RyanSkraba commented on code in PR #22010:
URL: https://github.com/apache/flink/pull/22010#discussion_r1200460171
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/SequenceGenerator.java:
##########
@@ -65,33 +71,40 @@ public void open(
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
+ ListStateDescriptor<InternalState> stateDescriptor =
+ new ListStateDescriptor<>(
+ name + "-sequence-state",
TypeInformation.of(InternalState.class));
+ this.checkpointedState =
context.getOperatorStateStore().getListState(stateDescriptor);
+ this.internalStates = Lists.newArrayList();
- for (Long v : this.checkpointedState.get()) {
- this.valuesToEmit.add(v);
- }
+ if (context.isRestored()) {
+ checkpointedState.get().forEach(state ->
internalStates.add(state));
} 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;
+ final long stepSize = runtimeContext.getNumberOfParallelSubtasks();
+ InternalState state = new InternalState(taskIdx, stepSize);
+ internalStates.add(state);
+ }
+ }
- for (long collected = 0; collected < toCollect; collected++) {
- this.valuesToEmit.add(collected * stepSize + congruence);
+ public Long nextValue() {
+ Iterator<InternalState> iterator = internalStates.iterator();
+ if (iterator.hasNext()) {
+ InternalState state = iterator.next();
+ long nextSequence =
Review Comment:
```suggestion
long nextSequence =
```
A really minor suggestion, but I'd inverse these variable names for
readability!
The one stored in the InternalState is the *next* or future value (to be
emitted on a later call): maybe `futureValue`?
The one currently being store in this local variable is the `currentValue`.
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/SequenceGenerator.java:
##########
@@ -65,33 +71,40 @@ public void open(
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
+ ListStateDescriptor<InternalState> stateDescriptor =
+ new ListStateDescriptor<>(
+ name + "-sequence-state",
TypeInformation.of(InternalState.class));
+ this.checkpointedState =
context.getOperatorStateStore().getListState(stateDescriptor);
+ this.internalStates = Lists.newArrayList();
- for (Long v : this.checkpointedState.get()) {
- this.valuesToEmit.add(v);
- }
+ if (context.isRestored()) {
+ checkpointedState.get().forEach(state ->
internalStates.add(state));
} 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;
+ final long stepSize = runtimeContext.getNumberOfParallelSubtasks();
+ InternalState state = new InternalState(taskIdx, stepSize);
+ internalStates.add(state);
+ }
+ }
- for (long collected = 0; collected < toCollect; collected++) {
- this.valuesToEmit.add(collected * stepSize + congruence);
+ public Long nextValue() {
+ Iterator<InternalState> iterator = internalStates.iterator();
+ if (iterator.hasNext()) {
+ InternalState state = iterator.next();
+ long nextSequence =
+ state.currentValue == 0 ? this.start + state.taskId :
state.currentValue;
+ state.currentValue = nextSequence + state.stepSize;
+ // All sequence values are cleared from the stateList after they
have been sent
+ if (state.currentValue > this.end) {
Review Comment:
If this.end is close to `Long.MAX_VALUE`, there's a possibility of overflow
here with `nextSequence + state.stepSize`.
```suggestion
if (state.currentValue > this.end || state.currentValue <=
nextSequence) {
```
(I think this is correct logic, but `Math.addExact` could be used as well.)
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/SequenceGenerator.java:
##########
@@ -65,33 +71,40 @@ public void open(
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
+ ListStateDescriptor<InternalState> stateDescriptor =
+ new ListStateDescriptor<>(
+ name + "-sequence-state",
TypeInformation.of(InternalState.class));
+ this.checkpointedState =
context.getOperatorStateStore().getListState(stateDescriptor);
+ this.internalStates = Lists.newArrayList();
- for (Long v : this.checkpointedState.get()) {
- this.valuesToEmit.add(v);
- }
+ if (context.isRestored()) {
+ checkpointedState.get().forEach(state ->
internalStates.add(state));
} 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;
+ final long stepSize = runtimeContext.getNumberOfParallelSubtasks();
+ InternalState state = new InternalState(taskIdx, stepSize);
+ internalStates.add(state);
+ }
+ }
- for (long collected = 0; collected < toCollect; collected++) {
- this.valuesToEmit.add(collected * stepSize + congruence);
+ public Long nextValue() {
+ Iterator<InternalState> iterator = internalStates.iterator();
+ if (iterator.hasNext()) {
+ InternalState state = iterator.next();
+ long nextSequence =
+ state.currentValue == 0 ? this.start + state.taskId :
state.currentValue;
Review Comment:
More seriously, there's a problem with using `state.currentValue == 0` as
the unitialized state. Imagine if you're dumping the range from -1000L to
1000L, when the task finds itself *meant* to be emitting zero, it will start
over from it's first value instead.
I'm working through it a bit, but it might be enough just to set it in the
InternalState from the start: `new InternalState(taskIdx, stepSize, start +
taskIdx)`.
--
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]