[
https://issues.apache.org/jira/browse/BEAM-9977?focusedWorklogId=450720&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-450720
]
ASF GitHub Bot logged work on BEAM-9977:
----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Jun/20 23:28
Start Date: 24/Jun/20 23:28
Worklog Time Spent: 10m
Work Description: boyuanzz commented on a change in pull request #11749:
URL: https://github.com/apache/beam/pull/11749#discussion_r445225947
##########
File path:
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -906,19 +1082,91 @@ public void setValueDeserializer(String
valueDeserializer) {
Coder<K> keyCoder = getKeyCoder(coderRegistry);
Coder<V> valueCoder = getValueCoder(coderRegistry);
- // Handles unbounded source to bounded conversion if maxNumRecords or
maxReadTime is set.
- Unbounded<KafkaRecord<K, V>> unbounded =
- org.apache.beam.sdk.io.Read.from(
-
toBuilder().setKeyCoder(keyCoder).setValueCoder(valueCoder).build().makeSource());
+ // The Read will be expanded into SDF transform when "beam_fn_api" is
enabled and
+ // "beam_fn_api_use_deprecated_read" is not enabled.
+ if (!ExperimentalOptions.hasExperiment(input.getPipeline().getOptions(),
"beam_fn_api")
+ || ExperimentalOptions.hasExperiment(
+ input.getPipeline().getOptions(),
"beam_fn_api_use_deprecated_read")) {
+ // Handles unbounded source to bounded conversion if maxNumRecords or
maxReadTime is set.
+ Unbounded<KafkaRecord<K, V>> unbounded =
+ org.apache.beam.sdk.io.Read.from(
+
toBuilder().setKeyCoder(keyCoder).setValueCoder(valueCoder).build().makeSource());
+
+ PTransform<PBegin, PCollection<KafkaRecord<K, V>>> transform =
unbounded;
+
+ if (getMaxNumRecords() < Long.MAX_VALUE || getMaxReadTime() != null) {
+ transform =
+
unbounded.withMaxReadTime(getMaxReadTime()).withMaxNumRecords(getMaxNumRecords());
+ }
- PTransform<PBegin, PCollection<KafkaRecord<K, V>>> transform = unbounded;
+ return input.getPipeline().apply(transform);
+ }
+ ReadAll<K, V> readTransform =
+ ReadAll.<K, V>read()
+ .withConsumerConfigOverrides(getConsumerConfig())
+ .withOffsetConsumerConfigOverrides(getOffsetConsumerConfig())
+ .withConsumerFactoryFn(getConsumerFactoryFn())
+ .withKeyDeserializerProvider(getKeyDeserializerProvider())
+ .withValueDeserializerProvider(getValueDeserializerProvider())
+ .withManualWatermarkEstimator()
+ .withTimestampPolicyFactory(getTimestampPolicyFactory());
+ if (isCommitOffsetsInFinalizeEnabled()) {
+ readTransform = readTransform.commitOffsets();
+ }
+ PCollection<KafkaSourceDescription> output =
+ input
+ .getPipeline()
+ .apply(Impulse.create())
+ .apply(ParDo.of(new GenerateKafkaSourceDescription(this)));
+ try {
+
output.setCoder(KafkaSourceDescription.getCoder(input.getPipeline().getSchemaRegistry()));
Review comment:
It works with `setSchema` but I want to make it explicitly because it's
possible that an user writes a DoFn which produces `KafkaSourceDescription`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 450720)
Time Spent: 18h 20m (was: 18h 10m)
> Build Kafka Read on top of Java SplittableDoFn
> ----------------------------------------------
>
> Key: BEAM-9977
> URL: https://issues.apache.org/jira/browse/BEAM-9977
> Project: Beam
> Issue Type: New Feature
> Components: io-java-kafka
> Reporter: Boyuan Zhang
> Assignee: Boyuan Zhang
> Priority: P2
> Time Spent: 18h 20m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)