dpcollins-google commented on a change in pull request #17103:
URL: https://github.com/apache/beam/pull/17103#discussion_r831471520



##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscriptionPartitionLoader.java
##########
@@ -47,15 +46,99 @@
   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<Void, SubscriptionPartition> {
+    @ProcessElement
+    public ProcessContinuation processElement(
+        OutputReceiver<SubscriptionPartition> output,
+        RestrictionTracker<Integer, Integer> restrictionTracker) {
+      int previousCount = restrictionTracker.currentRestriction();
+      int newCount = getPartitionCount.apply(topic);
+      if (newCount <= previousCount) {
+        return ProcessContinuation.resume().withResumeDelay(pollDuration);
+      }
+      if (!restrictionTracker.tryClaim(newCount)) {
+        return ProcessContinuation.stop();
+      }
+      Instant ts = previousCount == 0 ? BoundedWindow.TIMESTAMP_MIN_VALUE : 
getWatermark();
+      for (int i = previousCount; i < newCount; ++i) {
+        output.outputWithTimestamp(SubscriptionPartition.of(subscription, 
Partition.of(i)), ts);
+      }
+      if (terminate.get()) {
+        return ProcessContinuation.stop();
+      }
+      return ProcessContinuation.resume().withResumeDelay(pollDuration);
+    }
+
+    @GetInitialRestriction
+    public Integer getInitialRestriction() {
+      return 0;
+    }
+
+    @NewTracker
+    public RestrictionTracker<Integer, Integer> newTracker(@Restriction 
Integer input) {
+      return new RestrictionTracker<Integer, Integer>() {
+        private int position = input;
+
+        @Override
+        public boolean tryClaim(Integer newPosition) {
+          checkArgument(newPosition > position);
+          position = newPosition;
+          return true;
+        }
+
+        @Override
+        public Integer currentRestriction() {
+          return position;
+        }
+
+        @Override
+        public @Nullable SplitResult<Integer> trySplit(double 
fractionOfRemainder) {
+          return null;
+        }
+
+        @Override
+        public void checkDone() throws IllegalStateException {}

Review comment:
       I can't since I need to terminate regardless of if there is new data. 
Also, terminate is only used for testing.




-- 
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