Class KafkaIO.ReadSourceDescriptors<K,V>
- java.lang.Object
-
- org.apache.beam.sdk.transforms.PTransform<org.apache.beam.sdk.values.PCollection<KafkaSourceDescriptor>,org.apache.beam.sdk.values.PCollection<KafkaRecord<K,V>>>
-
- org.apache.beam.sdk.io.kafka.KafkaIO.ReadSourceDescriptors<K,V>
-
- All Implemented Interfaces:
java.io.Serializable,org.apache.beam.sdk.transforms.display.HasDisplayData
- Enclosing class:
- KafkaIO
public abstract static class KafkaIO.ReadSourceDescriptors<K,V> extends org.apache.beam.sdk.transforms.PTransform<org.apache.beam.sdk.values.PCollection<KafkaSourceDescriptor>,org.apache.beam.sdk.values.PCollection<KafkaRecord<K,V>>>
APTransformto read fromKafkaSourceDescriptor. SeeKafkaIOfor more information on usage and configuration. SeeReadFromKafkaDoFnfor more implementation details.During expansion, if
isCommitOffsetEnabled()istrue, the transform will expand to:
. Note that this expansion is not supported when running with x-lang on Dataflow.PCollection<KafkaSourceDescriptor> --> ParDo(ReadFromKafkaDoFn<KafkaSourceDescriptor, KV<KafkaSourceDescriptor, KafkaRecord>>) --> Map(output KafkaRecord) | --> KafkaCommitOffset- See Also:
- Serialized Form
-
-
Constructor Summary
Constructors Constructor Description ReadSourceDescriptors()
-
Method Summary
All Methods Static Methods Instance Methods Concrete Methods Modifier and Type Method Description KafkaIO.ReadSourceDescriptors<K,V>commitOffsets()Enable committing record offset.org.apache.beam.sdk.values.PCollection<KafkaRecord<K,V>>expand(org.apache.beam.sdk.values.PCollection<KafkaSourceDescriptor> input)static <K,V>
KafkaIO.ReadSourceDescriptors<K,V>read()KafkaIO.ReadSourceDescriptors<K,V>withAllowDuplicates()KafkaIO.ReadSourceDescriptors<K,V>withBadRecordErrorHandler(org.apache.beam.sdk.transforms.errorhandling.ErrorHandler<org.apache.beam.sdk.transforms.errorhandling.BadRecord,?> errorHandler)KafkaIO.ReadSourceDescriptors<K,V>withBootstrapServers(java.lang.String bootstrapServers)Sets the bootstrap servers to use for the Kafka consumer if unspecified via KafkaSourceDescriptor#getBootStrapServers()}.KafkaIO.ReadSourceDescriptors<K,V>withCheckStopReadingFn(@Nullable CheckStopReadingFn checkStopReadingFn)A customCheckStopReadingFnthat determines whether theReadFromKafkaDoFnshould stop reading from the givenTopicPartition.KafkaIO.ReadSourceDescriptors<K,V>withCheckStopReadingFn(@Nullable org.apache.beam.sdk.transforms.SerializableFunction<org.apache.kafka.common.TopicPartition,java.lang.Boolean> checkStopReadingFn)A customSerializableFunctionthat determines whether theReadFromKafkaDoFnshould stop reading from the givenTopicPartition.KafkaIO.ReadSourceDescriptors<K,V>withConsumerConfigOverrides(java.util.Map<java.lang.String,java.lang.Object> consumerConfig)Replaces the configuration for the main consumer.KafkaIO.ReadSourceDescriptors<K,V>withConsumerConfigUpdates(java.util.Map<java.lang.String,java.lang.Object> configUpdates)Updates configuration for the main consumer.KafkaIO.ReadSourceDescriptors<K,V>withConsumerFactoryFn(org.apache.beam.sdk.transforms.SerializableFunction<java.util.Map<java.lang.String,java.lang.Object>,org.apache.kafka.clients.consumer.Consumer<byte[],byte[]>> consumerFactoryFn)A factory to create KafkaConsumerfrom consumer configuration.KafkaIO.ReadSourceDescriptors<K,V>withConsumerPollingTimeout(long duration)Sets the timeout time in seconds for Kafka consumer polling request in theReadFromKafkaDoFn.KafkaIO.ReadSourceDescriptors<K,V>withCreateTime()Use the creation time ofKafkaRecordas the output timestamp.KafkaIO.ReadSourceDescriptors<K,V>withCreatWatermarkEstimatorFn(org.apache.beam.sdk.transforms.SerializableFunction<org.joda.time.Instant,org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator<org.joda.time.Instant>> fn)A function to create aWatermarkEstimator.KafkaIO.ReadSourceDescriptors<K,V>withExtractOutputTimestampFn(org.apache.beam.sdk.transforms.SerializableFunction<KafkaRecord<K,V>,org.joda.time.Instant> fn)A function to calculate output timestamp for a givenKafkaRecord.KafkaIO.ReadSourceDescriptors<K,V>withKeyDeserializer(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<K>> keyDeserializer)Sets a KafkaDeserializerto interpret key bytes read from Kafka.KafkaIO.ReadSourceDescriptors<K,V>withKeyDeserializerAndCoder(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<K>> keyDeserializer, org.apache.beam.sdk.coders.Coder<K> keyCoder)Sets a KafkaDeserializerfor interpreting key bytes read from Kafka along with aCoderfor helping the Beam runner materialize key objects at runtime if necessary.KafkaIO.ReadSourceDescriptors<K,V>withKeyDeserializerProvider(@Nullable DeserializerProvider<K> deserializerProvider)KafkaIO.ReadSourceDescriptors<K,V>withLogAppendTime()Use the log append time as the output timestamp.KafkaIO.ReadSourceDescriptors<K,V>withManualWatermarkEstimator()Use theWatermarkEstimators.Manualas the watermark estimator.KafkaIO.ReadSourceDescriptors<K,V>withMonotonicallyIncreasingWatermarkEstimator()Use theWatermarkEstimators.MonotonicallyIncreasingas the watermark estimator.KafkaIO.ReadSourceDescriptors<K,V>withOffsetConsumerConfigOverrides(@Nullable java.util.Map<java.lang.String,java.lang.Object> offsetConsumerConfig)Set additional configuration for the offset consumer.KafkaIO.ReadSourceDescriptors<K,V>withProcessingTime()Use the processing time as the output timestamp.KafkaIO.ReadSourceDescriptors<K,V>withReadCommitted()Sets "isolation_level" to "read_committed" in Kafka consumer configuration.KafkaIO.ReadSourceDescriptors<K,V>withRedistribute()Enable Redistribute.KafkaIO.ReadSourceDescriptors<K,V>withRedistributeNumKeys(int redistributeNumKeys)KafkaIO.ReadSourceDescriptors<K,V>withValueDeserializer(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<V>> valueDeserializer)Sets a KafkaDeserializerto interpret value bytes read from Kafka.KafkaIO.ReadSourceDescriptors<K,V>withValueDeserializerAndCoder(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<V>> valueDeserializer, org.apache.beam.sdk.coders.Coder<V> valueCoder)Sets a KafkaDeserializerfor interpreting value bytes read from Kafka along with aCoderfor helping the Beam runner materialize value objects at runtime if necessary.KafkaIO.ReadSourceDescriptors<K,V>withValueDeserializerProvider(@Nullable DeserializerProvider<V> deserializerProvider)KafkaIO.ReadSourceDescriptors<K,V>withWallTimeWatermarkEstimator()Use theWatermarkEstimators.WallTimeas the watermark estimator.-
Methods inherited from class org.apache.beam.sdk.transforms.PTransform
addAnnotation, compose, compose, getAdditionalInputs, getAnnotations, getDefaultOutputCoder, getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, getResourceHints, populateDisplayData, setDisplayData, setResourceHints, toString, validate, validate
-
-
-
-
Method Detail
-
read
public static <K,V> KafkaIO.ReadSourceDescriptors<K,V> read()
-
withBootstrapServers
public KafkaIO.ReadSourceDescriptors<K,V> withBootstrapServers(java.lang.String bootstrapServers)
Sets the bootstrap servers to use for the Kafka consumer if unspecified via KafkaSourceDescriptor#getBootStrapServers()}.
-
withKeyDeserializerProvider
public KafkaIO.ReadSourceDescriptors<K,V> withKeyDeserializerProvider(@Nullable DeserializerProvider<K> deserializerProvider)
-
withValueDeserializerProvider
public KafkaIO.ReadSourceDescriptors<K,V> withValueDeserializerProvider(@Nullable DeserializerProvider<V> deserializerProvider)
-
withKeyDeserializer
public KafkaIO.ReadSourceDescriptors<K,V> withKeyDeserializer(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<K>> keyDeserializer)
Sets a KafkaDeserializerto interpret key bytes read from Kafka.In addition, Beam also needs a
Coderto serialize and deserialize key objects at runtime. KafkaIO tries to infer a coder for the key based on theDeserializerclass, however in case that fails, you can usewithKeyDeserializerAndCoder(Class, Coder)to provide the key coder explicitly.
-
withValueDeserializer
public KafkaIO.ReadSourceDescriptors<K,V> withValueDeserializer(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<V>> valueDeserializer)
Sets a KafkaDeserializerto interpret value bytes read from Kafka.In addition, Beam also needs a
Coderto serialize and deserialize value objects at runtime. KafkaIO tries to infer a coder for the value based on theDeserializerclass, however in case that fails, you can usewithValueDeserializerAndCoder(Class, Coder)to provide the value coder explicitly.
-
withKeyDeserializerAndCoder
public KafkaIO.ReadSourceDescriptors<K,V> withKeyDeserializerAndCoder(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<K>> keyDeserializer, org.apache.beam.sdk.coders.Coder<K> keyCoder)
Sets a KafkaDeserializerfor interpreting key bytes read from Kafka along with aCoderfor helping the Beam runner materialize key objects at runtime if necessary.Use this method to override the coder inference performed within
withKeyDeserializer(Class).
-
withValueDeserializerAndCoder
public KafkaIO.ReadSourceDescriptors<K,V> withValueDeserializerAndCoder(java.lang.Class<? extends org.apache.kafka.common.serialization.Deserializer<V>> valueDeserializer, org.apache.beam.sdk.coders.Coder<V> valueCoder)
Sets a KafkaDeserializerfor interpreting value bytes read from Kafka along with aCoderfor helping the Beam runner materialize value objects at runtime if necessary.Use this method to override the coder inference performed within
withValueDeserializer(Class).
-
withConsumerFactoryFn
public KafkaIO.ReadSourceDescriptors<K,V> withConsumerFactoryFn(org.apache.beam.sdk.transforms.SerializableFunction<java.util.Map<java.lang.String,java.lang.Object>,org.apache.kafka.clients.consumer.Consumer<byte[],byte[]>> consumerFactoryFn)
A factory to create KafkaConsumerfrom consumer configuration. This is useful for supporting another version of Kafka consumer. Default isKafkaConsumer.
-
withCheckStopReadingFn
public KafkaIO.ReadSourceDescriptors<K,V> withCheckStopReadingFn(@Nullable CheckStopReadingFn checkStopReadingFn)
A customCheckStopReadingFnthat determines whether theReadFromKafkaDoFnshould stop reading from the givenTopicPartition.
-
withCheckStopReadingFn
public KafkaIO.ReadSourceDescriptors<K,V> withCheckStopReadingFn(@Nullable org.apache.beam.sdk.transforms.SerializableFunction<org.apache.kafka.common.TopicPartition,java.lang.Boolean> checkStopReadingFn)
A customSerializableFunctionthat determines whether theReadFromKafkaDoFnshould stop reading from the givenTopicPartition.
-
withConsumerConfigUpdates
public KafkaIO.ReadSourceDescriptors<K,V> withConsumerConfigUpdates(java.util.Map<java.lang.String,java.lang.Object> configUpdates)
Updates configuration for the main consumer. This method merges updates from the provided map with any prior updates usingKafkaIOUtils.DEFAULT_CONSUMER_PROPERTIESas the starting configuration.In
ReadFromKafkaDoFn, there're two consumers running in the backend:- the main consumer which reads data from kafka.
- the secondary offset consumer which is used to estimate the backlog by fetching the latest offset.
See
withConsumerConfigOverrides(java.util.Map<java.lang.String, java.lang.Object>)for overriding the configuration instead of updating it.See
withOffsetConsumerConfigOverrides(java.util.Map<java.lang.String, java.lang.Object>)for configuring the secondary offset consumer.
-
withExtractOutputTimestampFn
public KafkaIO.ReadSourceDescriptors<K,V> withExtractOutputTimestampFn(org.apache.beam.sdk.transforms.SerializableFunction<KafkaRecord<K,V>,org.joda.time.Instant> fn)
A function to calculate output timestamp for a givenKafkaRecord. The default value iswithProcessingTime().
-
withCreatWatermarkEstimatorFn
public KafkaIO.ReadSourceDescriptors<K,V> withCreatWatermarkEstimatorFn(org.apache.beam.sdk.transforms.SerializableFunction<org.joda.time.Instant,org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator<org.joda.time.Instant>> fn)
A function to create aWatermarkEstimator. The default value isWatermarkEstimators.MonotonicallyIncreasing.
-
withLogAppendTime
public KafkaIO.ReadSourceDescriptors<K,V> withLogAppendTime()
Use the log append time as the output timestamp.
-
withProcessingTime
public KafkaIO.ReadSourceDescriptors<K,V> withProcessingTime()
Use the processing time as the output timestamp.
-
withRedistribute
public KafkaIO.ReadSourceDescriptors<K,V> withRedistribute()
Enable Redistribute.
-
withAllowDuplicates
public KafkaIO.ReadSourceDescriptors<K,V> withAllowDuplicates()
-
withRedistributeNumKeys
public KafkaIO.ReadSourceDescriptors<K,V> withRedistributeNumKeys(int redistributeNumKeys)
-
withCreateTime
public KafkaIO.ReadSourceDescriptors<K,V> withCreateTime()
Use the creation time ofKafkaRecordas the output timestamp.
-
withWallTimeWatermarkEstimator
public KafkaIO.ReadSourceDescriptors<K,V> withWallTimeWatermarkEstimator()
Use theWatermarkEstimators.WallTimeas the watermark estimator.
-
withMonotonicallyIncreasingWatermarkEstimator
public KafkaIO.ReadSourceDescriptors<K,V> withMonotonicallyIncreasingWatermarkEstimator()
Use theWatermarkEstimators.MonotonicallyIncreasingas the watermark estimator.
-
withManualWatermarkEstimator
public KafkaIO.ReadSourceDescriptors<K,V> withManualWatermarkEstimator()
Use theWatermarkEstimators.Manualas the watermark estimator.
-
withReadCommitted
public KafkaIO.ReadSourceDescriptors<K,V> withReadCommitted()
Sets "isolation_level" to "read_committed" in Kafka consumer configuration. This ensures that the consumer does not read uncommitted messages. Kafka version 0.11 introduced transactional writes. Applications requiring end-to-end exactly-once semantics should only read committed messages. See JavaDoc forKafkaConsumerfor more description.
-
commitOffsets
public KafkaIO.ReadSourceDescriptors<K,V> commitOffsets()
Enable committing record offset. IfConsumerConfig.ENABLE_AUTO_COMMIT_CONFIGis set together withcommitOffsets(),commitOffsets()will be ignored.
-
withOffsetConsumerConfigOverrides
public KafkaIO.ReadSourceDescriptors<K,V> withOffsetConsumerConfigOverrides(@Nullable java.util.Map<java.lang.String,java.lang.Object> offsetConsumerConfig)
Set additional configuration for the offset consumer. It may be required for a secured Kafka cluster, especially when you see similar WARN log messageexception while fetching latest offset for partition {}. will be retried.In
ReadFromKafkaDoFn, there are two consumers running in the backend:- the main consumer which reads data from kafka.
- the secondary offset consumer which is used to estimate the backlog by fetching the latest offset.
By default, offset consumer inherits the configuration from main consumer, with an auto-generated
ConsumerConfig.GROUP_ID_CONFIG. This may not work in a secured Kafka which requires additional configuration.See
withConsumerConfigUpdates(java.util.Map<java.lang.String, java.lang.Object>)for configuring the main consumer.
-
withConsumerConfigOverrides
public KafkaIO.ReadSourceDescriptors<K,V> withConsumerConfigOverrides(java.util.Map<java.lang.String,java.lang.Object> consumerConfig)
Replaces the configuration for the main consumer.In
ReadFromKafkaDoFn, there are two consumers running in the backend:- the main consumer which reads data from kafka.
- the secondary offset consumer which is used to estimate the backlog by fetching the latest offset.
By default, main consumer uses the configuration from
KafkaIOUtils.DEFAULT_CONSUMER_PROPERTIES.See
withConsumerConfigUpdates(java.util.Map<java.lang.String, java.lang.Object>)for updating the configuration instead of overriding it.
-
withBadRecordErrorHandler
public KafkaIO.ReadSourceDescriptors<K,V> withBadRecordErrorHandler(org.apache.beam.sdk.transforms.errorhandling.ErrorHandler<org.apache.beam.sdk.transforms.errorhandling.BadRecord,?> errorHandler)
-
withConsumerPollingTimeout
public KafkaIO.ReadSourceDescriptors<K,V> withConsumerPollingTimeout(long duration)
Sets the timeout time in seconds for Kafka consumer polling request in theReadFromKafkaDoFn. A lower timeout optimizes for latency. Increase the timeout if the consumer is not fetching any records. The default is 2 seconds.
-
expand
public org.apache.beam.sdk.values.PCollection<KafkaRecord<K,V>> expand(org.apache.beam.sdk.values.PCollection<KafkaSourceDescriptor> input)
- Specified by:
expandin classorg.apache.beam.sdk.transforms.PTransform<org.apache.beam.sdk.values.PCollection<KafkaSourceDescriptor>,org.apache.beam.sdk.values.PCollection<KafkaRecord<K,V>>>
-
-