-
Notifications
You must be signed in to change notification settings - Fork 127
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-29398][connector/kafka] Provide rack ID to Kafka Source to take advantage of Rack Awareness #20
Conversation
- Added validation for the correctness of the rack ID supplied
Update docummentation to include rack awareness
Update docummentation to include rack awareness
Add RackId Example
remove validation session and add it to general rackID. Add client_rack documentation, Add sentence to link rackID to client.rack
FLINK-29398 Provide rack ID to Kafka Source to take advantage of Rack Awareness
Thanks for opening this pull request! Please check out our contributing guidelines. (https://flink.apache.org/contributing/how-to-contribute.html) |
Thanks for the PR @jeremy-degroot. I'll try to find some time to review this over this week. |
this.subtaskId = context.getIndexOfSubtask(); | ||
this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics; | ||
Properties consumerProps = new Properties(); | ||
consumerProps.putAll(props); | ||
consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props)); | ||
setConsumerClientRack(consumerProps, rackIdSupplier); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can this be resolved in the caller (KafkaSource) and ConsumerConfig.CLIENT_RACK_CONFIG
can be set there. The benefit would be that you do not need to change this file.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I looked into this, and while it's certainly possible to do it that way the testing path is much more complex. Since the Supplier would have to be resolved in another Supplier that's passed to the Reader, testing that the behavior is as expected in the actual execution path is difficult. In KafkaPartitionSplitReader, we can call the constructor directly and verify that the rackIdSupplier is called, and then also verify it does what we need it to by verifying the behavior of the helper method you noted further down.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah KafkaSource couldn't resolve the rack since the rack may not be the same for JM and TM. I would suggest to resolve this at the KafkaSourceReader if possible and avoid this low level change.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think it can be done in KafkaSourceReader either, since that doesn't interact with the Supplier at all. The alternative seems to be to resolve it in KafkaSourceFetcherManager, since that extends SingleThreadFetcherManager, where both Suppliers eventually run. But that would involve a rather messy overriding of the createSplitFetcher() method from that included class, and I don't think that's any better.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for the change @jeremy-degroot, I have made some minor comments
properties, | ||
UnregisteredMetricsGroup.createSourceReaderMetricGroup(), | ||
rackIdSupplier); | ||
reader.setConsumerClientRack(properties, rackIdSupplier); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Supplier function should get called from KafkaPartitionSplitReader
constructor. Do you need to call this method. Same for line 365
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this call is not needed, you can make the implementation of setConsumerClientRack
private.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As far as I know, neither the KafkaConsumer nor the KafkaPartitionSplitReader offers any way to examine the final consumer Properties, so to test the behavior of setConsumerClientRack we have to test it directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You can expose a method @VisibleForTesting or use reflection?
.../test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java
Outdated
Show resolved
Hide resolved
@@ -98,20 +98,24 @@ | |||
private final KafkaRecordDeserializationSchema<OUT> deserializationSchema; | |||
// The configurations. | |||
private final Properties props; | |||
// Client rackId callback | |||
private final Supplier<String> rackIdSupplier; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be a "CloseableSerializableSupplier". A supplier may contain a http client for example and also needs to be closed properly by Flink
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You're right about the Serializable bit, good catch.
Is there a straightforward way to do that while still allowing for Lambdas? Maybe provide one method for the basic Supplier and another for ClosableSerializableSupplier for when you need that degree of safety.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we must address this before we merge this PR.
At the very least, the Supplier
needs to be serializable.
You can use the SerializableSupplier
interface from Flink.
...onnector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java
Outdated
Show resolved
Hide resolved
Co-authored-by: Mason Chen <[email protected]>
- Changed the name of setRackId to setRackIdSupplier
# Conflicts: # docs/content/docs/connectors/datastream/kafka.md # flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java
@jeremy-degroot - this capability is one that we are currently in need of. Any idea when this might be approved/merged/released? Or are there other alternatives we might use to achieve rack awareness in flink? |
@mas-chen - any chance this will be reviewed/approved soon? |
I need this change as well, and it appears it wont be going into any release any time soon. I tried to extend the classes to add this functionality, but due to the excessive package/private access modifiers its not possible. Any suggestions on how I may do that without having to fork this project? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like there's two remaining items to address before we can merge this:
- The rack ID supplier needs to be at least
Serializable
.Closeable
variant can probably be added later. - Move the lazy configuration of rack id on client properties to be higher in the initialization call stacks - as it is right now, it's happening at too low-level (at the split reader).
@@ -98,20 +98,24 @@ | |||
private final KafkaRecordDeserializationSchema<OUT> deserializationSchema; | |||
// The configurations. | |||
private final Properties props; | |||
// Client rackId callback | |||
private final Supplier<String> rackIdSupplier; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we must address this before we merge this PR.
At the very least, the Supplier
needs to be serializable.
You can use the SerializableSupplier
interface from Flink.
… in KafkaSource - Changed the type of the KafkaSourceBuilder constructor parameter to a String - Changed the type of rackIdSupplier to a SerializableSupplier
…ry Supplier - Added test cases verifying the rack ID supplier is called at the right time - Simplified the tests in KafkaPartitionSplitReaderTest
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @jeremy-degroot! Just two minor nitpicks, I'll address them when I merge this.
Merging now ...
|
||
KafkaSource( | ||
KafkaSubscriber subscriber, | ||
OffsetsInitializer startingOffsetsInitializer, | ||
@Nullable OffsetsInitializer stoppingOffsetsInitializer, | ||
Boundedness boundedness, | ||
KafkaRecordDeserializationSchema<OUT> deserializationSchema, | ||
Properties props) { | ||
Properties props, | ||
SerializableSupplier<String> rackIdSupplier) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SerializableSupplier<String> rackIdSupplier) { | |
@Nullable SerializableSupplier<String> rackIdSupplier) { |
public KafkaSourceBuilder<OUT> setRackIdSupplier(SerializableSupplier<String> rackIdCallback) { | ||
this.rackIdSupplier = rackIdCallback; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: can we unify the naming here to just refer to this as the rackIdSupplier
?
Actually, @jeremy-degroot would you be able to squash all your changes into one commit that is rebased on latest |
…k Awareness Co-authored-by: jcmejias1 <[email protected]> Co-authored-by: Mason Chen <[email protected]> Co-authored-by: Ethan Gouty <[email protected]> Co-authored-by: Jeremy DeGroot <[email protected]> Co-authored-by: Siva Venkat Gogineni <[email protected]> This closes apache#53. This closes apache#20.
…k Awareness This closes apache#53. This closes apache#20. Co-authored-by: Jeremy DeGroot <[email protected]> Co-authored-by: jcmejias1 <[email protected]> Co-authored-by: Mason Chen <[email protected]> Co-authored-by: Ethan Gouty <[email protected]> Co-authored-by: Siva Venkat Gogineni <[email protected]>
What is the purpose of the change
This PR adds a new method to KafkaSourceBuilder that sets the
client.id
config for the KafkaSource to the value returned by the provided Supplier. It needs to be a Supplier because it needs to run on the TaskManager, and can't be determined at Job submit time like other configs.Brief change log
Verifying this change
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: yesDocumentation