gaoyunhaii commented on a change in pull request #17968:
URL: https://github.com/apache/flink/pull/17968#discussion_r761152475
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -560,71 +560,41 @@ protected void
processInput(MailboxDefaultAction.Controller controller) throws E
new
ResumeWrapper(controller.suspendDefaultAction(timer), timer)));
}
- protected void endData() throws Exception {
+ protected void endData(StopMode drain) throws Exception {
Review comment:
Perhaps rename drain to `mode` and remove the empty line?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
##########
@@ -69,17 +69,17 @@
private final AtomicBoolean stopped = new AtomicBoolean(false);
private enum FinishingReason {
- END_OF_DATA(true),
- STOP_WITH_SAVEPOINT_DRAIN(true),
- STOP_WITH_SAVEPOINT_NO_DRAIN(false);
+ END_OF_DATA(StopMode.DRAINED),
+ STOP_WITH_SAVEPOINT_DRAIN(StopMode.DRAINED),
+ STOP_WITH_SAVEPOINT_NO_DRAIN(StopMode.NO_DRAIN);
- private final boolean shouldCallFinish;
+ private final StopMode shouldCallFinish;
Review comment:
Rename `shouldCallFinish` to `stopMode` ?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
##########
@@ -251,12 +251,8 @@ protected void createInputProcessor(
},
"stop chained Flip-27 source for stop-with-savepoint --drain");
- return assertTriggeringCheckpointExceptions(
- sourcesStopped.thenCompose(
- ignore ->
- triggerSourcesCheckpointAsync(
- checkpointMetaData,
checkpointOptions)),
- checkpointMetaData.getCheckpointId());
+ return sourcesStopped.thenCompose(
Review comment:
Hi Dawid I might not fully understand the change since it seems
FLINK-23471 does not change the behavior of triggering checkpoint, and only
modifies the behavior of `notifyCheckpointComplete / notifyCheckpointAbort` ?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StopMode.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.runtime.tasks;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+/**
+ * Tells if the job is stopping because of consuming all data. {@link
#DRAINED} means the job is
+ * stopping either with a stop-with-savepoint --drain or due to consuming all
records in the source.
+ * A drained pipeline should call {@link StreamOperator#finish()} on all
operators.
+ */
+@Internal
+public enum StopMode {
Review comment:
Hi Dawid do you think we might keep using `boolean shouldDrain`? I'm
asking since currently it seems we have a mix use of the enum and the boolean,
and have transformation on received `CheckpointBarrier` or emit `EndOfData`.
But I'm might not be very strongly argue for that.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamOperatorWrapper.java
##########
@@ -120,18 +120,19 @@ void setNext(StreamOperatorWrapper next) {
* MailboxExecutor#yield()} to take the mails of closing operator and
running timers and run
* them.
*/
- public void finish(StreamTaskActionExecutor actionExecutor) throws
Exception {
- if (!isHead) {
+ public void finish(StreamTaskActionExecutor actionExecutor, StopMode
stopMode)
Review comment:
There is also a reference in the class document to this method that
needs to add one more parameter
--
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]