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

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

                Author: ASF GitHub Bot
            Created on: 15/May/20 18:49
            Start Date: 15/May/20 18:49
    Worklog Time Spent: 10m 
      Work Description: boyuanzz commented on a change in pull request #11715:
URL: https://github.com/apache/beam/pull/11715#discussion_r425986350



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.transforms.splittabledofn;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.io.range.OffsetRange;
+
+/**
+ * A special {@link OffsetRangeTracker} for tracking a growable offset range. 
The Long.MAX_VALUE is
+ * used as end range to indicate the possibility of infinity.
+ *
+ * <p>A offset range is considered as growable when the end offset could 
grow(or change) during
+ * execution time(e.g., Kafka backlog, appended file).
+ */
+@Experimental(Kind.SPLITTABLE_DO_FN)
+public class GrowableOffsetRangeTracker extends OffsetRangeTracker {
+  /**
+   * An interface that should be implemented to fetch estimated end offset of 
range.
+   *
+   * <p>{@code estimateRangeEnd} is called to give te end offset when {@code 
trySplit} or {@code
+   * getProgress} is invoked. The end offset is exclusive for the range. It's 
not necessary to
+   * increase monotonically but it's only taken into computation when it's 
larger than the current
+   * position. When returning Long.MAX_VALUE as estimate, it means the largest 
possible position for
+   * the range is Long.MAX_VALUE - 1. Having a good estimate is important for 
providing a good
+   * signal of progress and splitting at a proper position.
+   */
+  public interface OffsetPoller {
+    long estimateRangeEnd();
+  }
+
+  private final OffsetPoller poller;
+
+  public GrowableOffsetRangeTracker(long start, OffsetPoller offsetPoller) {
+    super(new OffsetRange(start, Long.MAX_VALUE));
+    this.poller = checkNotNull(offsetPoller);
+  }
+
+  @Override
+  public SplitResult<OffsetRange> trySplit(double fractionOfRemainder) {
+    // If current tracking range is no longer growable, split it as a normal 
range.
+    if (range.getTo() != Long.MAX_VALUE || range.getTo() == range.getFrom()) {
+      return super.trySplit(fractionOfRemainder);
+    }
+    long cur = (lastAttemptedOffset == null) ? range.getFrom() - 1 : 
lastAttemptedOffset;
+
+    // If current range has been done, there is no more space to split.
+    if (cur == Long.MAX_VALUE) {
+      return null;
+    }
+    // Fetch the estimated end offset. If the estimated end is smaller than 
the next offset, use
+    // the next offset as end.
+    long estimateLatestOffset = Long.max(poller.estimateRangeEnd(), cur + 1);
+    long splitPos =
+        cur
+            + Math.max(
+                1L,
+                (Double.valueOf((estimateLatestOffset - cur) * 
fractionOfRemainder)).longValue());
+
+    if (splitPos > estimateLatestOffset) {
+      return null;
+    }
+    OffsetRange res = new OffsetRange(splitPos, range.getTo());
+    this.range = new OffsetRange(range.getFrom(), splitPos);
+    return SplitResult.of(range, res);

Review comment:
       There is a slight difference around comparison of `splitPos` and `end`.  
`GrowableOffsetTracker` allows split when `splitPos <= end` but 
`OffsetRangeTracker` allows split when `splitPos < end`. The purpose of 
`OffsetRangeTracker` is to reduce resuming from an empty range. The purpose of 
`GrowableOffsetRangeTracker` is to handle split at `cur + 1`.

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.transforms.splittabledofn;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.io.range.OffsetRange;
+
+/**
+ * A special {@link OffsetRangeTracker} for tracking a growable offset range. 
The Long.MAX_VALUE is
+ * used as end range to indicate the possibility of infinity.
+ *
+ * <p>A offset range is considered as growable when the end offset could 
grow(or change) during
+ * execution time(e.g., Kafka backlog, appended file).
+ */
+@Experimental(Kind.SPLITTABLE_DO_FN)
+public class GrowableOffsetRangeTracker extends OffsetRangeTracker {
+  /**
+   * An interface that should be implemented to fetch estimated end offset of 
range.
+   *
+   * <p>{@code estimateRangeEnd} is called to give te end offset when {@code 
trySplit} or {@code
+   * getProgress} is invoked. The end offset is exclusive for the range. It's 
not necessary to
+   * increase monotonically but it's only taken into computation when it's 
larger than the current
+   * position. When returning Long.MAX_VALUE as estimate, it means the largest 
possible position for
+   * the range is Long.MAX_VALUE - 1. Having a good estimate is important for 
providing a good
+   * signal of progress and splitting at a proper position.
+   */
+  public interface OffsetPoller {
+    long estimateRangeEnd();
+  }
+
+  private final OffsetPoller poller;
+
+  public GrowableOffsetRangeTracker(long start, OffsetPoller offsetPoller) {
+    super(new OffsetRange(start, Long.MAX_VALUE));
+    this.poller = checkNotNull(offsetPoller);
+  }
+
+  @Override
+  public SplitResult<OffsetRange> trySplit(double fractionOfRemainder) {
+    // If current tracking range is no longer growable, split it as a normal 
range.
+    if (range.getTo() != Long.MAX_VALUE || range.getTo() == range.getFrom()) {

Review comment:
       Yes `super.trySplit()` handles empty range. The special case here is to 
handle `range.getTo() == range.getFrom == Long.MAX_VALUE`.




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


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

    Worklog Id:     (was: 433840)
    Time Spent: 50m  (was: 40m)

> 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: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>




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

Reply via email to