gaoyunhaii commented on a change in pull request #18805:
URL: https://github.com/apache/flink/pull/18805#discussion_r818868121



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperator.java
##########
@@ -117,7 +142,16 @@ public void initializeState(StateInitializationContext 
context) throws Exception
         restoredCheckpointId = checkpointId;
         InitContext initContext =
                 createInitContext(checkpointId.isPresent() ? 
checkpointId.getAsLong() : null);
-
+        if (committableSerializer != null) {
+            final ListState<List<CommT>> legacyCommitterState =

Review comment:
       Might move this statement into `context.isRestored`

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkV1Adapter.java
##########
@@ -447,5 +451,13 @@ public void commit(Collection<CommitRequest<CommT>> 
committables)
                 committables.forEach(CommitRequest::retryLater);

Review comment:
       Here seems should be `failures.forEach`? 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/SinkWriterOperator.java
##########
@@ -185,12 +219,36 @@ private void emitCommittables(Long checkpointId) throws 
IOException, Interrupted
         Collection<CommT> committables =
                 ((PrecommittingSinkWriter<?, CommT>) 
sinkWriter).prepareCommit();
         StreamingRuntimeContext runtimeContext = getRuntimeContext();
-        int indexOfThisSubtask = runtimeContext.getIndexOfThisSubtask();
+        final int indexOfThisSubtask = runtimeContext.getIndexOfThisSubtask();
+        final int numberOfParallelSubtasks = 
runtimeContext.getNumberOfParallelSubtasks();
+
+        // Emit only committable summary if there are legacy committables
+        if (!legacyCommittables.isEmpty()) {
+            emit(

Review comment:
       Might add `checkState(checkpointId > InitContext.INITIAL_CHECKPOINT_ID)`?




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