[ 
https://issues.apache.org/jira/browse/BEAM-9977?focusedWorklogId=451684&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-451684
 ]

ASF GitHub Bot logged work on BEAM-9977:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 26/Jun/20 18:37
            Start Date: 26/Jun/20 18:37
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on a change in pull request 
#11749:
URL: https://github.com/apache/beam/pull/11749#discussion_r446348275



##########
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:
       I don't think you should need `setSchema` either, we should always use 
the SchemaCoder for a PCollection<T> where T has a Schema registered (unless 
the PCollection has a specific coder set, or T also has a default coder set):
   
   
https://github.com/apache/beam/blob/b83c06d47bd5e2e2905599297981353af234b034/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java#L154-L166
   
   I'm not sure I follow why a DoFn that produces `KafkaSourceDescription` 
requires any special logic




----------------------------------------------------------------
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: 451684)
    Time Spent: 19.5h  (was: 19h 20m)

> 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: 19.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to