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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to