scwhittle commented on code in PR #36849:
URL: https://github.com/apache/beam/pull/36849#discussion_r2541066980


##########
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/KafkaReadWithRedistributeOverride.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow;
+
+import java.util.Map;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
+import org.apache.beam.sdk.io.kafka.KafkaRecord;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.runners.PTransformMatcher;
+import org.apache.beam.sdk.runners.PTransformOverrideFactory;
+import org.apache.beam.sdk.util.construction.ReplacementOutputs;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+@SuppressWarnings({
+  "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
+  "nullness", // TODO(https://github.com/apache/beam/issues/20497)
+  "PatternMatchingInstanceof"
+})
+public final class KafkaReadWithRedistributeOverride {
+
+  private KafkaReadWithRedistributeOverride() {}
+
+  public static PTransformMatcher matcher() {
+    return new PTransformMatcher() {
+      @Override
+      public boolean matches(AppliedPTransform<?, ?, ?> application) {
+        if (application.getTransform() instanceof KafkaIO.Read) {
+          return ((KafkaIO.Read) application.getTransform()).isRedistributed();
+        }
+        return false;
+      }
+    };
+  }
+
+  /**
+   * {@link PTransformOverrideFactory} for {@link KafkaIO.Read} that enables 
{@code
+   * withOffsetDeduplication} when {@code withRedistribute} is enabled.
+   */
+  static class Factory<K, V>
+      implements PTransformOverrideFactory<
+          PBegin, PCollection<KafkaRecord<K, V>>, KafkaIO.Read<K, V>> {
+
+    @Override
+    public PTransformReplacement<PBegin, PCollection<KafkaRecord<K, V>>> 
getReplacementTransform(
+        AppliedPTransform<PBegin, PCollection<KafkaRecord<K, V>>, 
KafkaIO.Read<K, V>> transform) {
+      KafkaIO.Read<K, V> read = transform.getTransform();
+      return PTransformReplacement.of(
+          transform.getPipeline().begin(), read.withOffsetDeduplication(true));

Review Comment:
   can we determine if the user has explicitly disabled offsetdeduplication by 
setting it to false and not override the default to true in that case?



##########
runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/KafkaReadWithRedistributeOverrideTest.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+
+import java.io.Serializable;
+import java.util.Collections;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
+import org.apache.beam.sdk.runners.PTransformOverride;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class KafkaReadWithRedistributeOverrideTest implements Serializable {
+  @Rule public transient TestPipeline p = TestPipeline.create();
+
+  @Test
+  public void testOverrideAppliedWhenRedistributeEnabled() {
+    p.apply(
+        "MatchingRead",
+        KafkaIO.<String, String>read()
+            .withBootstrapServers("localhost:9092")
+            .withTopic("test_match")
+            .withKeyDeserializer(StringDeserializer.class)
+            .withValueDeserializer(StringDeserializer.class)
+            .withRedistribute());

Review Comment:
   would also be good to cover explicit enable of delegated and explicit 
disable in addition to unspecified



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to