Class KafkaSourceBuilder<OUT>


  • @PublicEvolving
    public class KafkaSourceBuilder<OUT>
    extends Object
    The builder class for KafkaSource to make it easier for the users to construct a KafkaSource.

    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_UNBOUNDED mode and never stops until the Flink job is canceled or fails. To let the KafkaSource run as Boundedness.CONTINUOUS_UNBOUNDED yet stop at some given offsets, one can call setUnbounded(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())
         .setRackId(() -> MY_RACK_ID)
         .build();
     

    Check the Java docs of each individual methods to learn more about the settings to build a KafkaSource.

    • 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 use setTopicPattern(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 use setTopicPattern(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 java Pattern.
        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 - the KafkaSubscriber to use for split discovery.
        Returns:
        this KafkaSourceBuilder.
      • setUnbounded

        public KafkaSourceBuilder<OUT> setUnbounded​(OffsetsInitializer stoppingOffsetsInitializer)
        By default the KafkaSource is set to run as Boundedness.CONTINUOUS_UNBOUNDED and 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 an OffsetsInitializer to 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 return Boundedness.CONTINUOUS_UNBOUNDED even though it will stop at the stopping offsets specified by the stopping offsets OffsetsInitializer.

        The following OffsetsInitializer are commonly used and provided out of the box. Users can also implement their own OffsetsInitializer for 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 whose ConsumerRecord.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 - The OffsetsInitializer to 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 as Boundedness.CONTINUOUS_UNBOUNDED and thus never stops until the Flink job fails or is canceled. To let the KafkaSource run as Boundedness.BOUNDED and stop at some point, one can set an OffsetsInitializer to 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 return Boundedness.BOUNDED instead of Boundedness.CONTINUOUS_UNBOUNDED.

        The following OffsetsInitializer are commonly used and provided out of the box. Users can also implement their own OffsetsInitializer for 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 whose ConsumerRecord.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 - the OffsetsInitializer to specify the stopping offsets.
        Returns:
        this KafkaSourceBuilder.
        See Also:
        setUnbounded(OffsetsInitializer)
      • setValueOnlyDeserializer

        public KafkaSourceBuilder<OUT> setValueOnlyDeserializer​(org.apache.flink.api.common.serialization.DeserializationSchema<OUT> deserializationSchema)
        Sets the deserializer of the ConsumerRecord for KafkaSource. The given DeserializationSchema will be used to deserialize the value of ConsumerRecord. The other information (e.g. key) in a ConsumerRecord will be ignored.
        Parameters:
        deserializationSchema - the DeserializationSchema to 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.
      • setRackIdSupplier

        public KafkaSourceBuilder<OUT> setRackIdSupplier​(org.apache.flink.util.function.SerializableSupplier<String> rackIdCallback)
        Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader.
        Parameters:
        rackIdCallback - callback to provide Kafka consumer client.rack
        Returns:
        this KafkaSourceBuilder
      • setProperties

        public KafkaSourceBuilder<OUT> setProperties​(Properties props)
        Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found in ConsumerConfig and KafkaSourceOptions.

        Note that the following keys will be overridden by the builder when the KafkaSource is created.

        Parameters:
        props - the properties to set for the KafkaSource.
        Returns:
        this KafkaSourceBuilder.
      • build

        public KafkaSource<OUT> build()
        Build the KafkaSource.
        Returns:
        a KafkaSource with the settings made for this builder.