beam-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (Jira)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-10123) Create CommitOffsetTransform to commit Kafka record offset
Date Tue, 29 Sep 2020 23:12:00 GMT

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

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

                Author: ASF GitHub Bot
            Created on: 29/Sep/20 23:11
            Start Date: 29/Sep/20 23:11
    Worklog Time Spent: 10m 
      Work Description: boyuanzz commented on a change in pull request #12572:
URL: https://github.com/apache/beam/pull/12572#discussion_r497125319



##########
File path: sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCommitOffset.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.Preconditions.checkState;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.Max;
+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.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A {@link PTransform} that commits offsets of {@link KafkaRecord}. */
+public class KafkaCommitOffset<K, V>
+    extends PTransform<
+        PCollection<KV<KafkaSourceDescriptor, KafkaRecord<K, V>>>, PCollection<Void>>
{
+  private final KafkaIO.ReadSourceDescriptors<K, V> readSourceDescriptors;
+
+  KafkaCommitOffset(KafkaIO.ReadSourceDescriptors<K, V> readSourceDescriptors) {
+    this.readSourceDescriptors = readSourceDescriptors;
+  }
+
+  static class CommitOffsetDoFn extends DoFn<KV<KafkaSourceDescriptor, Long>, Void>
{
+    private static final Logger LOG = LoggerFactory.getLogger(CommitOffsetDoFn.class);
+    private final Map<String, Object> offsetConsumerConfig;
+    private final Map<String, Object> consumerConfig;
+    private final SerializableFunction<Map<String, Object>, Consumer<byte[],
byte[]>>
+        consumerFactoryFn;
+
+    private transient ConsumerSpEL consumerSpEL = null;
+
+    CommitOffsetDoFn(KafkaIO.ReadSourceDescriptors readSourceDescriptors) {
+      offsetConsumerConfig = readSourceDescriptors.getOffsetConsumerConfig();
+      consumerConfig = readSourceDescriptors.getConsumerConfig();
+      consumerFactoryFn = readSourceDescriptors.getConsumerFactoryFn();
+    }
+
+    @ProcessElement
+    public void processElement(@Element KV<KafkaSourceDescriptor, Long> element) {
+      Map<String, Object> updatedConsumerConfig =
+          overrideBootstrapServersConfig(consumerConfig, element.getKey());
+      try (Consumer<byte[], byte[]> offsetConsumer =
+          consumerFactoryFn.apply(
+              KafkaIOUtils.getOffsetConsumerConfig(
+                  "commitOffset", offsetConsumerConfig, updatedConsumerConfig))) {
+        try {
+          offsetConsumer.commitSync(
+              Collections.singletonMap(
+                  element.getKey().getTopicPartition(),
+                  new OffsetAndMetadata(element.getValue() + 1)));
+        } catch (Exception e) {
+          // TODO: consider retrying.
+          LOG.warn("Getting exception when committing offset: {}", e.getMessage());
+        }
+      }
+    }
+
+    private Map<String, Object> overrideBootstrapServersConfig(
+        Map<String, Object> currentConfig, KafkaSourceDescriptor description) {
+      checkState(
+          currentConfig.containsKey(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)
+              || description.getBootStrapServers() != null);
+      Map<String, Object> config = new HashMap<>(currentConfig);
+      if (description.getBootStrapServers() != null
+          && description.getBootStrapServers().size() > 0) {
+        config.put(
+            ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
+            String.join(",", description.getBootStrapServers()));
+      }
+      return config;
+    }
+  }
+
+  @Override
+  public PCollection<Void> expand(PCollection<KV<KafkaSourceDescriptor, KafkaRecord<K,
V>>> input) {
+    try {
+      return input
+          .apply(
+              MapElements.into(new TypeDescriptor<KV<KafkaSourceDescriptor, Long>>()
{})
+                  .via(element -> KV.of(element.getKey(), element.getValue().getOffset())))
+          .setCoder(
+              KvCoder.of(
+                  input
+                      .getPipeline()
+                      .getSchemaRegistry()
+                      .getSchemaCoder(KafkaSourceDescriptor.class),
+                  VarLongCoder.of()))
+          .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))))

Review comment:
       That's my expectation of the time interval for committing. The reason for committing
offset is to have a good start point when we restart the pipeline, so it not requires a real-time
commtting. Do you have any suggestion on this time?




----------------------------------------------------------------
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:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 492714)
    Time Spent: 1.5h  (was: 1h 20m)

> Create CommitOffsetTransform to commit Kafka record offset
> ----------------------------------------------------------
>
>                 Key: BEAM-10123
>                 URL: https://issues.apache.org/jira/browse/BEAM-10123
>             Project: Beam
>          Issue Type: Sub-task
>          Components: io-java-kafka
>            Reporter: Boyuan Zhang
>            Assignee: Boyuan Zhang
>            Priority: P3
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>




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

Mime
View raw message