pabloem commented on code in PR #21752: URL: https://github.com/apache/beam/pull/21752#discussion_r908780163
########## sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/WatchForKafkaTopicPartitions.java: ########## @@ -0,0 +1,184 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.beam.sdk.annotations.Experimental; +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; + +/** + * A {@link PTransform} for continuously querying Kafka for new partitions, and emitting those + * topics as {@link KafkaSourceDescriptor} This transform is implemented using the {@link Watch} + * transform, and modifications to this transform should keep that in mind. + * + * <p>Please see + * https://docs.google.com/document/d/1Io49s5LBs29HJyppKG3AlR-gHz5m5PC6CqO0CCoSqLs/edit?usp=sharing + * for design details + */ +@Experimental +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 = firstNonNull(checkDuration, DEFAULT_CHECK_DURATION); + 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(kafkaConsumerFactoryFn, kafkaConsumerConfig, topics)) + .withPollInterval(checkDuration)) + .apply(ParDo.of(new ConvertToDescriptor(checkStopReadingFn, startReadTime, stopReadTime))); + } + + private static class ConvertToDescriptor + extends DoFn<KV<byte[], TopicPartition>, KafkaSourceDescriptor> { + + private final @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn; + private final @Nullable Instant startReadTime; + private final @Nullable Instant stopReadTime; + + private ConvertToDescriptor( + @Nullable SerializableFunction<TopicPartition, Boolean> checkStopReadingFn, + @Nullable Instant startReadTime, + @Nullable Instant stopReadTime) { + this.checkStopReadingFn = checkStopReadingFn; + this.startReadTime = startReadTime; + this.stopReadTime = stopReadTime; + } + + @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(); Review Comment: fyi - runner metrics systems are not designed to support 'big data'. Dataflow only guarantees a few thousands of metrics being supported (which I guess is enough in this case?) so just worth keeping in mind. -- 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]
