You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/22 15:39:18 UTC

[GitHub] [beam] Abacn commented on a diff in pull request #21752: Feature/beam 13852 reimplement with dynamic read

Abacn commented on code in PR #21752:
URL: https://github.com/apache/beam/pull/21752#discussion_r903851736


##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.sdk.io.kafka;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.Watch.Growth.PollFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+public class WatchForKafkaTopicPartitions

Review Comment:
   The original WatchKafkaTopicPartitionDoFn was package private and here this is a public transform. Is it necessary? if decided to introducing a new public ptransform, WatchForKafkaTopicPartitions class should also comes with javadoc.



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.sdk.io.kafka;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.Watch.Growth.PollFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+public class WatchForKafkaTopicPartitions
+    extends PTransform<PBegin, PCollection<KafkaSourceDescriptor>> {
+
+  private static final Duration DEFAULT_CHECK_DURATION = Duration.standardHours(1);
+  private static final String COUNTER_NAMESPACE = "watch_kafka_topic_partition";
+
+  private final Duration checkDuration;
+  private final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
+      kafkaConsumerFactoryFn;
+  private final Map<String, Object> kafkaConsumerConfig;
+  private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn;
+  private final List<String> topics;
+  private final @Nullable Instant startReadTime;
+  private final @Nullable Instant stopReadTime;
+
+  public WatchForKafkaTopicPartitions(
+      @Nullable Duration checkDuration,
+      SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> kafkaConsumerFactoryFn,
+      Map<String, Object> kafkaConsumerConfig,
+      @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn,
+      List<String> topics,
+      @Nullable Instant startReadTime,
+      @Nullable Instant stopReadTime) {
+    this.checkDuration = checkDuration == null ? DEFAULT_CHECK_DURATION : checkDuration;

Review Comment:
   minor: could use firstNonNull



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.sdk.io.kafka;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.Watch.Growth.PollFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+public class WatchForKafkaTopicPartitions
+    extends PTransform<PBegin, PCollection<KafkaSourceDescriptor>> {
+
+  private static final Duration DEFAULT_CHECK_DURATION = Duration.standardHours(1);
+  private static final String COUNTER_NAMESPACE = "watch_kafka_topic_partition";
+
+  private final Duration checkDuration;
+  private final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
+      kafkaConsumerFactoryFn;
+  private final Map<String, Object> kafkaConsumerConfig;
+  private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn;
+  private final List<String> topics;
+  private final @Nullable Instant startReadTime;
+  private final @Nullable Instant stopReadTime;
+
+  public WatchForKafkaTopicPartitions(
+      @Nullable Duration checkDuration,
+      SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> kafkaConsumerFactoryFn,
+      Map<String, Object> kafkaConsumerConfig,
+      @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn,
+      List<String> topics,
+      @Nullable Instant startReadTime,
+      @Nullable Instant stopReadTime) {
+    this.checkDuration = checkDuration == null ? DEFAULT_CHECK_DURATION : checkDuration;
+    this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn;
+    this.kafkaConsumerConfig = kafkaConsumerConfig;
+    this.checkStopReadingFn = checkStopReadingFn;
+    this.topics = topics;
+    this.startReadTime = startReadTime;
+    this.stopReadTime = stopReadTime;
+  }
+
+  @Override
+  public PCollection<KafkaSourceDescriptor> expand(PBegin input) {
+    return input
+        .apply(Impulse.create())
+        .apply(
+            "Match new TopicPartitions",
+            Watch.growthOf(new WatchPartitionFn()).withPollInterval(checkDuration))
+        .apply(ParDo.of(new ConvertToDescriptor()));
+  }
+
+  private class ConvertToDescriptor

Review Comment:
   Consider make inner class static per the guideline of DoFn [here](https://github.com/apache/beam/blob/adac54e1fa4f8f5915b47b16fa7b343b29304c07/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java#L298). May need to change reference-to-outer class to constructor params.



##########
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java:
##########
@@ -114,6 +130,17 @@ public class KafkaIOIT {
 
   @Rule public TestPipeline readPipeline = TestPipeline.create();
 
+  private static ExperimentalOptions pipelineOptions;

Review Comment:
   consider naming `sdfPipelineOptions` since it is used only by `sdfReadPipeline` and put the initialization in setUp



##########
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java:
##########
@@ -259,8 +272,131 @@ public void processElement(
               return null;
             });
 
+    PipelineResult writeResult = writePipeline.run();
+    writeResult.waitUntilFinish();
+
     PipelineResult readResult = readPipeline.run();
-    readResult.waitUntilFinish();
+    PipelineResult.State readState =
+        readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout()));
+
+    cancelIfTimeouted(readResult, readState);
+  }
+
+  @Test
+  public void testKafkaWithDynamicPartitions() throws IOException {
+    AdminClient client =
+        AdminClient.create(
+            ImmutableMap.of("bootstrap.servers", options.getKafkaBootstrapServerAddresses()));
+    String topicName = "DynamicTopicPartition-" + UUID.randomUUID();
+    Map<Integer, String> records = new HashMap<>();
+    for (int i = 0; i < 100; i++) {
+      records.put(i, String.valueOf(i));
+    }
+    Map<Integer, String> moreRecords = new HashMap<>();
+    for (int i = 100; i < 200; i++) {
+      moreRecords.put(i, String.valueOf(i));
+    }
+    try {
+      client.createTopics(ImmutableSet.of(new NewTopic(topicName, 1, (short) 1)));
+      client.createPartitions(ImmutableMap.of(topicName, NewPartitions.increaseTo(1)));
+
+      writePipeline
+          .apply("Generate Write Elements", Create.of(records))
+          .apply(
+              "Write to Kafka",
+              KafkaIO.<Integer, String>write()
+                  .withBootstrapServers(options.getKafkaBootstrapServerAddresses())
+                  .withTopic(topicName)
+                  .withKeySerializer(IntegerSerializer.class)
+                  .withValueSerializer(StringSerializer.class));
+
+      writePipeline.run().waitUntilFinish(Duration.standardSeconds(15));
+
+      System.out.println("after write 1");
+
+      Thread delayedWriteThread =
+          new Thread(
+              () -> {
+                try {
+                  Thread.sleep(20 * 1000); // wait 20 seconds before changing kafka

Review Comment:
   I was experiencing flakes using Thread inside tests because wait time is often out of control in different test machines. What is the purpose of waiting 20 seconds here, and is it robust?



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.sdk.io.kafka;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.transforms.Watch.Growth.PollFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+public class WatchForKafkaTopicPartitions
+    extends PTransform<PBegin, PCollection<KafkaSourceDescriptor>> {
+
+  private static final Duration DEFAULT_CHECK_DURATION = Duration.standardHours(1);
+  private static final String COUNTER_NAMESPACE = "watch_kafka_topic_partition";
+
+  private final Duration checkDuration;
+  private final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
+      kafkaConsumerFactoryFn;
+  private final Map<String, Object> kafkaConsumerConfig;
+  private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn;
+  private final List<String> topics;
+  private final @Nullable Instant startReadTime;
+  private final @Nullable Instant stopReadTime;
+
+  public WatchForKafkaTopicPartitions(
+      @Nullable Duration checkDuration,
+      SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> kafkaConsumerFactoryFn,
+      Map<String, Object> kafkaConsumerConfig,
+      @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn,
+      List<String> topics,
+      @Nullable Instant startReadTime,
+      @Nullable Instant stopReadTime) {
+    this.checkDuration = checkDuration == null ? DEFAULT_CHECK_DURATION : checkDuration;
+    this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn;
+    this.kafkaConsumerConfig = kafkaConsumerConfig;
+    this.checkStopReadingFn = checkStopReadingFn;
+    this.topics = topics;
+    this.startReadTime = startReadTime;
+    this.stopReadTime = stopReadTime;
+  }
+
+  @Override
+  public PCollection<KafkaSourceDescriptor> expand(PBegin input) {
+    return input
+        .apply(Impulse.create())
+        .apply(
+            "Match new TopicPartitions",
+            Watch.growthOf(new WatchPartitionFn()).withPollInterval(checkDuration))
+        .apply(ParDo.of(new ConvertToDescriptor()));
+  }
+
+  private class ConvertToDescriptor
+      extends DoFn<KV<byte[], TopicPartition>, KafkaSourceDescriptor> {
+    @ProcessElement
+    public void processElement(
+        @Element KV<byte[], TopicPartition> partition,
+        OutputReceiver<KafkaSourceDescriptor> receiver) {
+      TopicPartition topicPartition = Objects.requireNonNull(partition.getValue());
+      if (checkStopReadingFn == null || !checkStopReadingFn.apply(topicPartition)) {
+        Counter foundedTopicPartition =
+            Metrics.counter(COUNTER_NAMESPACE, topicPartition.toString());
+        foundedTopicPartition.inc();
+        receiver.output(
+            KafkaSourceDescriptor.of(
+                topicPartition, null, startReadTime, null, stopReadTime, null));
+      }
+    }
+  }
+
+  private class WatchPartitionFn extends PollFn<byte[], TopicPartition> {

Review Comment:
   consider static class also here



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org