JingsongLi commented on a change in pull request #12074:
URL: https://github.com/apache/flink/pull/12074#discussion_r424893175



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/SequenceGenerator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.flink.streaming.api.functions.source.datagen;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/**
+ * A stateful, re-scalable {@link DataGenerator} that emits each number from a 
given interval
+ * exactly once, possibly in parallel.
+ */
+@Experimental
+public abstract class SequenceGenerator<T> implements DataGenerator<T> {
+
+       private final long start;
+       private final long end;
+
+       private transient ListState<Long> checkpointedState;
+       protected transient Deque<Long> valuesToEmit;
+
+       /**
+        * Creates a DataGenerator that emits all numbers from the given 
interval exactly once.
+        *
+        * @param start Start of the range of numbers to emit.
+        * @param end End of the range of numbers to emit.
+        */
+       public SequenceGenerator(long start, long end) {
+               this.start = start;
+               this.end = end;
+       }
+
+       @Override
+       public void open(
+                       String name,
+                       FunctionInitializationContext context,
+                       RuntimeContext runtimeContext) throws Exception {
+               Preconditions.checkState(this.checkpointedState == null,
+                               "The " + getClass().getSimpleName() + " has 
already been initialized.");
+
+               this.checkpointedState = 
context.getOperatorStateStore().getListState(
+                               new ListStateDescriptor<>(
+                                               name + "-sequence-state",
+                                               LongSerializer.INSTANCE));
+               this.valuesToEmit = new ArrayDeque<>();
+               if (context.isRestored()) {
+                       // upon restoring
+
+                       for (Long v : this.checkpointedState.get()) {
+                               this.valuesToEmit.add(v);
+                       }
+               } else {
+                       // the first time the job is executed
+                       final int stepSize = 
runtimeContext.getNumberOfParallelSubtasks();
+                       final int taskIdx = 
runtimeContext.getIndexOfThisSubtask();
+                       final long congruence = start + taskIdx;
+
+                       long totalNoOfElements = Math.abs(end - start + 1);
+                       final int baseSize = safeDivide(totalNoOfElements, 
stepSize);
+                       final int toCollect = (totalNoOfElements % stepSize > 
taskIdx) ? baseSize + 1 : baseSize;
+
+                       for (long collected = 0; collected < toCollect; 
collected++) {

Review comment:
       Yes, this is the same mechanism to `StatefulSequenceSource`, it is hard 
to modify, we can leave this improvement to future.




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


Reply via email to