Class KafkaSourceBuilder<OUT>
- java.lang.Object
-
- org.apache.flink.connector.kafka.source.KafkaSourceBuilder<OUT>
-
@PublicEvolving public class KafkaSourceBuilder<OUT> extends Object
The builder class forKafkaSourceto make it easier for the users to construct aKafkaSource.The following example shows the minimum setup to create a KafkaSource that reads the String values from a Kafka topic.
KafkaSource<String> source = KafkaSource .<String>builder() .setBootstrapServers(MY_BOOTSTRAP_SERVERS) .setTopics(Arrays.asList(TOPIC1, TOPIC2)) .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) .build();The bootstrap servers, topics/partitions to consume, and the record deserializer are required fields that must be set.
To specify the starting offsets of the KafkaSource, one can call
setStartingOffsets(OffsetsInitializer).By default the KafkaSource runs in an
Boundedness.CONTINUOUS_UNBOUNDEDmode and never stops until the Flink job is canceled or fails. To let the KafkaSource run asBoundedness.CONTINUOUS_UNBOUNDEDyet stop at some given offsets, one can callsetUnbounded(OffsetsInitializer). For example the following KafkaSource stops after it consumes up to the latest partition offsets at the point when the Flink job started.KafkaSource<String> source = KafkaSource .<String>builder() .setBootstrapServers(MY_BOOTSTRAP_SERVERS) .setTopics(Arrays.asList(TOPIC1, TOPIC2)) .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) .setUnbounded(OffsetsInitializer.latest()) .build();Check the Java docs of each individual methods to learn more about the settings to build a KafkaSource.
-
-
Field Summary
Fields Modifier and Type Field Description protected Propertiesprops
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description KafkaSource<OUT>build()Build theKafkaSource.KafkaSourceBuilder<OUT>setBootstrapServers(String bootstrapServers)Sets the bootstrap servers for the KafkaConsumer of the KafkaSource.KafkaSourceBuilder<OUT>setBounded(OffsetsInitializer stoppingOffsetsInitializer)By default the KafkaSource is set to run asBoundedness.CONTINUOUS_UNBOUNDEDand thus never stops until the Flink job fails or is canceled.KafkaSourceBuilder<OUT>setClientIdPrefix(String prefix)Sets the client id prefix of this KafkaSource.KafkaSourceBuilder<OUT>setDeserializer(KafkaRecordDeserializationSchema<OUT> recordDeserializer)Sets thedeserializerof theConsumerRecordfor KafkaSource.KafkaSourceBuilder<OUT>setGroupId(String groupId)Sets the consumer group id of the KafkaSource.KafkaSourceBuilder<OUT>setKafkaSubscriber(KafkaSubscriber kafkaSubscriber)Set a custom Kafka subscriber to use to discover new splits.KafkaSourceBuilder<OUT>setPartitions(Set<org.apache.kafka.common.TopicPartition> partitions)Set a set of partitions to consume from.KafkaSourceBuilder<OUT>setProperties(Properties props)Set arbitrary properties for the KafkaSource and KafkaConsumer.KafkaSourceBuilder<OUT>setProperty(String key, String value)Set an arbitrary property for the KafkaSource and KafkaConsumer.KafkaSourceBuilder<OUT>setStartingOffsets(OffsetsInitializer startingOffsetsInitializer)Specify from which offsets the KafkaSource should start consuming from by providing anOffsetsInitializer.KafkaSourceBuilder<OUT>setTopicPattern(Pattern topicPattern)Set a topic pattern to consume from use the javaPattern.KafkaSourceBuilder<OUT>setTopics(String... topics)Set a list of topics the KafkaSource should consume from.KafkaSourceBuilder<OUT>setTopics(List<String> topics)Set a list of topics the KafkaSource should consume from.KafkaSourceBuilder<OUT>setUnbounded(OffsetsInitializer stoppingOffsetsInitializer)By default the KafkaSource is set to run asBoundedness.CONTINUOUS_UNBOUNDEDand thus never stops until the Flink job fails or is canceled.KafkaSourceBuilder<OUT>setValueOnlyDeserializer(org.apache.flink.api.common.serialization.DeserializationSchema<OUT> deserializationSchema)Sets thedeserializerof theConsumerRecordfor KafkaSource.
-
-
-
Field Detail
-
props
protected Properties props
-
-
Method Detail
-
setBootstrapServers
public KafkaSourceBuilder<OUT> setBootstrapServers(String bootstrapServers)
Sets the bootstrap servers for the KafkaConsumer of the KafkaSource.- Parameters:
bootstrapServers- the bootstrap servers of the Kafka cluster.- Returns:
- this KafkaSourceBuilder.
-
setGroupId
public KafkaSourceBuilder<OUT> setGroupId(String groupId)
Sets the consumer group id of the KafkaSource.- Parameters:
groupId- the group id of the KafkaSource.- Returns:
- this KafkaSourceBuilder.
-
setTopics
public KafkaSourceBuilder<OUT> setTopics(List<String> topics)
Set a list of topics the KafkaSource should consume from. All the topics in the list should have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of the topics to be created lazily, please usesetTopicPattern(Pattern)instead.- Parameters:
topics- the list of topics to consume from.- Returns:
- this KafkaSourceBuilder.
- See Also:
KafkaConsumer.subscribe(Collection)
-
setTopics
public KafkaSourceBuilder<OUT> setTopics(String... topics)
Set a list of topics the KafkaSource should consume from. All the topics in the list should have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of the topics to be created lazily, please usesetTopicPattern(Pattern)instead.- Parameters:
topics- the list of topics to consume from.- Returns:
- this KafkaSourceBuilder.
- See Also:
KafkaConsumer.subscribe(Collection)
-
setTopicPattern
public KafkaSourceBuilder<OUT> setTopicPattern(Pattern topicPattern)
Set a topic pattern to consume from use the javaPattern.- Parameters:
topicPattern- the pattern of the topic name to consume from.- Returns:
- this KafkaSourceBuilder.
- See Also:
KafkaConsumer.subscribe(Pattern)
-
setPartitions
public KafkaSourceBuilder<OUT> setPartitions(Set<org.apache.kafka.common.TopicPartition> partitions)
Set a set of partitions to consume from.- Parameters:
partitions- the set of partitions to consume from.- Returns:
- this KafkaSourceBuilder.
- See Also:
KafkaConsumer.assign(Collection)
-
setKafkaSubscriber
public KafkaSourceBuilder<OUT> setKafkaSubscriber(KafkaSubscriber kafkaSubscriber)
Set a custom Kafka subscriber to use to discover new splits.- Parameters:
kafkaSubscriber- theKafkaSubscriberto use for split discovery.- Returns:
- this KafkaSourceBuilder.
-
setStartingOffsets
public KafkaSourceBuilder<OUT> setStartingOffsets(OffsetsInitializer startingOffsetsInitializer)
Specify from which offsets the KafkaSource should start consuming from by providing anOffsetsInitializer.The following
OffsetsInitializers are commonly used and provided out of the box. Users can also implement their ownOffsetsInitializerfor custom behaviors.OffsetsInitializer.earliest()- starting from the earliest offsets. This is also the defaultOffsetsInitializerof the KafkaSource for starting offsets.OffsetsInitializer.latest()- starting from the latest offsets.OffsetsInitializer.committedOffsets()- starting from the committed offsets of the consumer group.OffsetsInitializer.committedOffsets(org.apache.kafka.clients.consumer.OffsetResetStrategy)- starting from the committed offsets of the consumer group. If there is no committed offsets, starting from the offsets specified by theOffsetResetStrategy.OffsetsInitializer.offsets(Map)- starting from the specified offsets for each partition.OffsetsInitializer.timestamp(long)- starting from the specified timestamp for each partition. Note that the guarantee here is that all the records in Kafka whoseConsumerRecord.timestamp()is greater than the given starting timestamp will be consumed. However, it is possible that some consumer records whose timestamp is smaller than the given starting timestamp are also consumed.
- Parameters:
startingOffsetsInitializer- theOffsetsInitializersetting the starting offsets for the Source.- Returns:
- this KafkaSourceBuilder.
-
setUnbounded
public KafkaSourceBuilder<OUT> setUnbounded(OffsetsInitializer stoppingOffsetsInitializer)
By default the KafkaSource is set to run asBoundedness.CONTINUOUS_UNBOUNDEDand thus never stops until the Flink job fails or is canceled. To let the KafkaSource run as a streaming source but still stop at some point, one can set anOffsetsInitializerto specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the KafkaSource will then exit.This method is different from
setBounded(OffsetsInitializer)in that after setting the stopping offsets with this method,KafkaSource.getBoundedness()will still returnBoundedness.CONTINUOUS_UNBOUNDEDeven though it will stop at the stopping offsets specified by the stopping offsetsOffsetsInitializer.The following
OffsetsInitializerare commonly used and provided out of the box. Users can also implement their ownOffsetsInitializerfor custom behaviors.OffsetsInitializer.latest()- stop at the latest offsets of the partitions when the KafkaSource starts to run.OffsetsInitializer.committedOffsets()- stops at the committed offsets of the consumer group.OffsetsInitializer.offsets(Map)- stops at the specified offsets for each partition.OffsetsInitializer.timestamp(long)- stops at the specified timestamp for each partition. The guarantee of setting the stopping timestamp is that no Kafka records whoseConsumerRecord.timestamp()is greater than the given stopping timestamp will be consumed. However, it is possible that some records whose timestamp is smaller than the specified stopping timestamp are not consumed.
- Parameters:
stoppingOffsetsInitializer- TheOffsetsInitializerto specify the stopping offset.- Returns:
- this KafkaSourceBuilder.
- See Also:
setBounded(OffsetsInitializer)
-
setBounded
public KafkaSourceBuilder<OUT> setBounded(OffsetsInitializer stoppingOffsetsInitializer)
By default the KafkaSource is set to run asBoundedness.CONTINUOUS_UNBOUNDEDand thus never stops until the Flink job fails or is canceled. To let the KafkaSource run asBoundedness.BOUNDEDand stop at some point, one can set anOffsetsInitializerto specify the stopping offsets for each partition. When all the partitions have reached their stopping offsets, the KafkaSource will then exit.This method is different from
setUnbounded(OffsetsInitializer)in that after setting the stopping offsets with this method,KafkaSource.getBoundedness()will returnBoundedness.BOUNDEDinstead ofBoundedness.CONTINUOUS_UNBOUNDED.The following
OffsetsInitializerare commonly used and provided out of the box. Users can also implement their ownOffsetsInitializerfor custom behaviors.OffsetsInitializer.latest()- stop at the latest offsets of the partitions when the KafkaSource starts to run.OffsetsInitializer.committedOffsets()- stops at the committed offsets of the consumer group.OffsetsInitializer.offsets(Map)- stops at the specified offsets for each partition.OffsetsInitializer.timestamp(long)- stops at the specified timestamp for each partition. The guarantee of setting the stopping timestamp is that no Kafka records whoseConsumerRecord.timestamp()is greater than the given stopping timestamp will be consumed. However, it is possible that some records whose timestamp is smaller than the specified stopping timestamp are not consumed.
- Parameters:
stoppingOffsetsInitializer- theOffsetsInitializerto specify the stopping offsets.- Returns:
- this KafkaSourceBuilder.
- See Also:
setUnbounded(OffsetsInitializer)
-
setDeserializer
public KafkaSourceBuilder<OUT> setDeserializer(KafkaRecordDeserializationSchema<OUT> recordDeserializer)
Sets thedeserializerof theConsumerRecordfor KafkaSource.- Parameters:
recordDeserializer- the deserializer for KafkaConsumerRecord.- Returns:
- this KafkaSourceBuilder.
-
setValueOnlyDeserializer
public KafkaSourceBuilder<OUT> setValueOnlyDeserializer(org.apache.flink.api.common.serialization.DeserializationSchema<OUT> deserializationSchema)
Sets thedeserializerof theConsumerRecordfor KafkaSource. The givenDeserializationSchemawill be used to deserialize the value of ConsumerRecord. The other information (e.g. key) in a ConsumerRecord will be ignored.- Parameters:
deserializationSchema- theDeserializationSchemato use for deserialization.- Returns:
- this KafkaSourceBuilder.
-
setClientIdPrefix
public KafkaSourceBuilder<OUT> setClientIdPrefix(String prefix)
Sets the client id prefix of this KafkaSource.- Parameters:
prefix- the client id prefix to use for this KafkaSource.- Returns:
- this KafkaSourceBuilder.
-
setProperty
public KafkaSourceBuilder<OUT> setProperty(String key, String value)
Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found inConsumerConfigandKafkaSourceOptions.Note that the following keys will be overridden by the builder when the KafkaSource is created.
key.deserializeris always set toByteArrayDeserializer.value.deserializeris always set toByteArrayDeserializer.auto.offset.reset.strategyis overridden byOffsetsInitializer.getAutoOffsetResetStrategy()for the starting offsets, which is by defaultOffsetsInitializer.earliest().partition.discovery.interval.msis overridden to -1 whensetBounded(OffsetsInitializer)has been invoked.
- Parameters:
key- the key of the property.value- the value of the property.- Returns:
- this KafkaSourceBuilder.
-
setProperties
public KafkaSourceBuilder<OUT> setProperties(Properties props)
Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found inConsumerConfigandKafkaSourceOptions.Note that the following keys will be overridden by the builder when the KafkaSource is created.
key.deserializeris always set toByteArrayDeserializer.value.deserializeris always set toByteArrayDeserializer.auto.offset.reset.strategyis overridden byOffsetsInitializer.getAutoOffsetResetStrategy()for the starting offsets, which is by defaultOffsetsInitializer.earliest().partition.discovery.interval.msis overridden to -1 whensetBounded(OffsetsInitializer)has been invoked.client.idis overridden to the "client.id.prefix-RANDOM_LONG", or "group.id-RANDOM_LONG" if the client id prefix is not set.
- Parameters:
props- the properties to set for the KafkaSource.- Returns:
- this KafkaSourceBuilder.
-
build
public KafkaSource<OUT> build()
Build theKafkaSource.- Returns:
- a KafkaSource with the settings made for this builder.
-
-