xintongsong commented on code in PR #24272:
URL: https://github.com/apache/flink/pull/24272#discussion_r1501806777


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorAttributesBuilder.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.operators;
+
+import org.apache.flink.annotation.Experimental;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+/** The builder class for {@link OperatorAttributes}. */
+@Experimental
+public class OperatorAttributesBuilder {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(OperatorAttributesBuilder.class);
+    @Nullable private Boolean outputOnlyAfterEndOfStream = null;

Review Comment:
   Agree with @mohitjain2504 's opinion. In this particular case, I don't think 
it's necessary to distinguish the two cases. Might be nicer to keep it simple.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java:
##########
@@ -31,19 +35,27 @@
 public class CheckpointCoordinatorDeActivator implements JobStatusListener {
 
     private final CheckpointCoordinator coordinator;
+    private final Map<JobVertexID, ExecutionJobVertex> tasks;
 
-    public CheckpointCoordinatorDeActivator(CheckpointCoordinator coordinator) 
{
+    public CheckpointCoordinatorDeActivator(
+            CheckpointCoordinator coordinator, Map<JobVertexID, 
ExecutionJobVertex> tasks) {
         this.coordinator = checkNotNull(coordinator);
+        this.tasks = checkNotNull(tasks);
     }
 
     @Override
     public void jobStatusChanges(JobID jobId, JobStatus newJobStatus, long 
timestamp) {
-        if (newJobStatus == JobStatus.RUNNING) {
-            // start the checkpoint scheduler
+        if (newJobStatus == JobStatus.RUNNING && allTasksOutputNonBlocking()) {
+            // start the checkpoint scheduler if there is no blocking edge
             coordinator.startCheckpointScheduler();
         } else {
             // anything else should stop the trigger for now
             coordinator.stopCheckpointScheduler();
         }
     }
+
+    private boolean allTasksOutputNonBlocking() {
+        return tasks.values().stream()
+                .noneMatch(vertex -> 
vertex.getJobVertex().isAnyOutputBlocking());

Review Comment:
   The value of this never changes. It should be calculated at initialization, 
rather than every time `jobStatusChanges` is called.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureReason.java:
##########
@@ -61,6 +61,8 @@ public enum CheckpointFailureReason {
 
     CHECKPOINT_COORDINATOR_SHUTDOWN(false, "CheckpointCoordinator shutdown."),
 
+    CHECKPOINT_COORDINATOR_START(false, "Checkpoint Coordinator is starting."),

Review Comment:
   The message is confusing.
   1. According to how it is used, it's not about starting the checkpoint 
coordinator, but about canceling potential prior timers before schedule a new 
one.
   2. I don't really get how could this happen? Is this comes from first stop 
the checkpoint scheduler and then start it again? Then shouldn't the timers be 
already canceled at stopping the checkpoint scheduler?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PhysicalTransformation.java:
##########
@@ -76,4 +76,8 @@ public void setSupportsConcurrentExecutionAttempts(
             boolean supportsConcurrentExecutionAttempts) {
         this.supportsConcurrentExecutionAttempts = 
supportsConcurrentExecutionAttempts;
     }
+
+    public boolean isOutputOnlyAfterEndOfStream() {
+        return false;

Review Comment:
   Let's add a comment indicating that this is just a default value, and is not 
by definition of PhysicalTransformation.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -75,7 +75,7 @@ public class StreamNode {
     private KeySelector<?, ?>[] statePartitioners = new KeySelector[0];
     private TypeSerializer<?> stateKeySerializer;
 
-    private StreamOperatorFactory<?> operatorFactory;
+    private @Nullable StreamOperatorFactory<?> operatorFactory;

Review Comment:
   Why this becomes nullable? How does it relates to exposing operator 
attributes?



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