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

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

                Author: ASF GitHub Bot
            Created on: 22/Mar/22 20:10
            Start Date: 22/Mar/22 20:10
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #17103:
URL: https://github.com/apache/beam/pull/17103#discussion_r832581769



##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionLoader.java
##########
@@ -47,15 +47,109 @@
   private final SubscriptionPath subscription;
   private final SerializableFunction<TopicPath, Integer> getPartitionCount;
   private final Duration pollDuration;
-  private final boolean terminate;
+  private final SerializableSupplier<Boolean> terminate;
+
+  private class GeneratorFn extends DoFn<byte[], SubscriptionPartition> {
+    @ProcessElement
+    public ProcessContinuation processElement(
+        RestrictionTracker<Integer, Integer> restrictionTracker,
+        OutputReceiver<SubscriptionPartition> output,
+        ManualWatermarkEstimator<Instant> estimator) {
+      int previousCount = restrictionTracker.currentRestriction();
+      int newCount = getPartitionCount.apply(topic);
+      if (!restrictionTracker.tryClaim(newCount)) {
+        return ProcessContinuation.stop();
+      }
+      if (newCount > previousCount) {
+        for (int i = previousCount; i < newCount; ++i) {
+          output.outputWithTimestamp(
+              SubscriptionPartition.of(subscription, Partition.of(i)),
+              estimator.currentWatermark());
+        }
+      }
+      estimator.setWatermark(getWatermark());
+      return ProcessContinuation.resume().withResumeDelay(pollDuration);
+    }
+
+    @GetInitialWatermarkEstimatorState
+    public Instant getInitialWatermarkEstimatorState(@Timestamp Instant 
initial) {
+      checkArgument(initial.equals(BoundedWindow.TIMESTAMP_MIN_VALUE));
+      return initial;
+    }
+
+    @GetInitialRestriction
+    public Integer getInitialRestriction() {
+      return 0;
+    }
+
+    @NewTracker
+    public RestrictionTracker<Integer, Integer> newTracker(@Restriction 
Integer input) {
+      return new RestrictionTracker<Integer, Integer>() {
+        private boolean terminated = false;
+        private int position = input;
+
+        @Override
+        public boolean tryClaim(Integer newPosition) {
+          checkArgument(newPosition >= position);
+          if (terminate.get()) {

Review comment:
       ```suggestion
             if (terminated) {
               return false;
             }
             if (terminate.get()) {
   ```
   
   `trySplit` may have already set `terminated` to true which means that future 
`tryClaim` calls should fail.




-- 
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-unsubscr...@beam.apache.org

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


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

    Worklog Id:     (was: 746086)
    Time Spent: 12h  (was: 11h 50m)

> Fix issues with Pub/Sub Lite IO at high volumes
> -----------------------------------------------
>
>                 Key: BEAM-14129
>                 URL: https://issues.apache.org/jira/browse/BEAM-14129
>             Project: Beam
>          Issue Type: Task
>          Components: io-java-gcp
>            Reporter: Daniel Collins
>            Assignee: Daniel Collins
>            Priority: P1
>             Fix For: 2.38.0
>
>          Time Spent: 12h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to