-
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
[FLINK-29398][connector/kafka] Provide rack ID to Kafka Source to take advantage of Rack Awareness #20
Changes from all commits
9c23f7d
b4aa761
83771ec
190e2e6
7c2cf26
cf218ad
ba451d9
d85ecd1
2a6aaa5
c6550c9
167098e
5a7aca7
a829ecf
c98b9f3
be13242
342cd72
d0b24f9
9acf6ba
91cfa27
3cf0794
b0d5d02
e3bd8bc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,7 @@ | |
import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator; | ||
import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; | ||
import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; | ||
import org.apache.flink.util.function.SerializableSupplier; | ||
|
||
import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
import org.apache.kafka.common.TopicPartition; | ||
|
@@ -80,6 +81,7 @@ | |
* .setTopics(Arrays.asList(TOPIC1, TOPIC2)) | ||
* .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) | ||
* .setUnbounded(OffsetsInitializer.latest()) | ||
* .setRackId(() -> MY_RACK_ID) | ||
* .build(); | ||
* }</pre> | ||
* | ||
|
@@ -100,6 +102,8 @@ public class KafkaSourceBuilder<OUT> { | |
private KafkaRecordDeserializationSchema<OUT> deserializationSchema; | ||
// The configurations. | ||
protected Properties props; | ||
// Client rackId supplier | ||
private SerializableSupplier<String> rackIdSupplier; | ||
|
||
KafkaSourceBuilder() { | ||
this.subscriber = null; | ||
|
@@ -108,6 +112,7 @@ public class KafkaSourceBuilder<OUT> { | |
this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; | ||
this.deserializationSchema = null; | ||
this.props = new Properties(); | ||
this.rackIdSupplier = null; | ||
} | ||
|
||
/** | ||
|
@@ -355,6 +360,17 @@ public KafkaSourceBuilder<OUT> setClientIdPrefix(String prefix) { | |
return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); | ||
} | ||
|
||
/** | ||
* Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader. | ||
* | ||
* @param rackIdCallback callback to provide Kafka consumer client.rack | ||
* @return this KafkaSourceBuilder | ||
*/ | ||
public KafkaSourceBuilder<OUT> setRackIdSupplier(SerializableSupplier<String> rackIdCallback) { | ||
this.rackIdSupplier = rackIdCallback; | ||
Comment on lines
+369
to
+370
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
return this; | ||
} | ||
|
||
/** | ||
* Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found | ||
* in {@link ConsumerConfig} and {@link KafkaSourceOptions}. | ||
|
@@ -422,7 +438,8 @@ public KafkaSource<OUT> build() { | |
stoppingOffsetsInitializer, | ||
boundedness, | ||
deserializationSchema, | ||
props); | ||
props, | ||
rackIdSupplier); | ||
} | ||
|
||
// ------------- private helpers -------------- | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -80,11 +80,20 @@ public KafkaPartitionSplitReader( | |
Properties props, | ||
SourceReaderContext context, | ||
KafkaSourceReaderMetrics kafkaSourceReaderMetrics) { | ||
this(props, context, kafkaSourceReaderMetrics, null); | ||
} | ||
|
||
public KafkaPartitionSplitReader( | ||
Properties props, | ||
SourceReaderContext context, | ||
KafkaSourceReaderMetrics kafkaSourceReaderMetrics, | ||
String rackIdSupplier) { | ||
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 commentThe reason will be displayed to describe this comment to others. Learn more. Can this be resolved in the caller (KafkaSource) and There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe 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 commentThe 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. |
||
this.consumer = new KafkaConsumer<>(consumerProps); | ||
this.stoppingOffsets = new HashMap<>(); | ||
this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG); | ||
|
@@ -256,6 +265,20 @@ KafkaConsumer<byte[], byte[]> consumer() { | |
|
||
// --------------- private helper method ---------------------- | ||
|
||
/** | ||
* This Method performs Null and empty Rack Id validation and sets the rack id to the | ||
* client.rack Consumer Config. | ||
* | ||
* @param consumerProps Consumer Property. | ||
* @param rackId Rack Id's. | ||
*/ | ||
@VisibleForTesting | ||
void setConsumerClientRack(Properties consumerProps, String rackId) { | ||
if (rackId != null && !rackId.isEmpty()) { | ||
consumerProps.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, rackId); | ||
} | ||
} | ||
|
||
private void parseStartingOffsets( | ||
KafkaPartitionSplit split, | ||
List<TopicPartition> partitionsStartingFromEarliest, | ||
|
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.