[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-15 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r248145667
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -746,23 +750,16 @@ public boolean checkInputDependencyConstraints() {
}
 
/**
-* An input is consumable when
-* 1. the source result is PIPELINED and one of the result partition 
has produced data.
-* 2. the source result is BLOCKING and is FINISHED(all partitions are 
FINISHED).
+* Get whether an input of the vertex is consumable.
+* An input is consumable when when any partition in it is consumable.
+*
+* Note that a BLOCKING result partition is only consumable when all 
partitions in the result are FINISHED.
 *
 * @return whether the input is consumable
 */
public boolean isInputConsumable(int inputNumber) {
 
 Review comment:
   Changed it to default scope as it will be invoked by `Execution` and tests.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-15 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r248145670
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -736,7 +740,7 @@ void sendPartitionInfos() {
 * @return whether the input constraint is satisfied
 */
public boolean checkInputDependencyConstraints() {
 
 Review comment:
   Changed it to default scope as it will be invoked by tests.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-15 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r248145662
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -689,7 +693,7 @@ void scheduleOrUpdateConsumers(ResultPartitionID 
partitionId) {
 
if 
(partition.getIntermediateResult().getResultType().isPipelined()) {
// Schedule or update receivers of this partition
-   partition.markSomePipelinedDataProduced();
+   partition.markDataProduced();
 
 Review comment:
   Sure,  `partition.markDataProduced` is fine for BLOCKING result. I moved it 
out from the `if` clause.
   
   In my understanding, the `ExecutionVertex.scheduleOrUpdateConsumers` is for 
PIPELINED partition currently. Only PIPELINED partition will send the 
`scheduleOrUpdateConsumers` message and the method will throw exception if the 
partition is BLOCKING. 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246435193
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +730,41 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
+   if (getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY) {
 
 Review comment:
   I've move `InputDependencyConstraint ` to JobVertex. And the job wide 
default value can be configured in `ExecutionConfig`. But I haven't make it 
configurable through DataSet/DataStream API yet.
   
   I agree we should support the constraint configurable for each operator. But 
I'm not quite sure whether we should support it with DataSet API or later for 
the stream/batch unified StreamGraph/Transformation API? Could you share your 
suggestion?
   
   In our production experience, a job-wide configured input constraint 
satisfies most users, together with `BATCH_FORCED` execution mode, to ensure a 
batch job can finish with limited resources.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246435193
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +730,41 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
+   if (getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY) {
 
 Review comment:
   I've move `InputDependencyConstraint ` to JobVertex. And the job wide 
default value can be configured in `ExecutionConfig`. But I haven't make it 
configurable through DataSet/DataStream API yet.
   
   I agree we should support the constraint configurable for each operator. But 
I'm not quite sure whether we should support it with DataSet API or later for 
the unified StreamGraph/Transformation API? Could you share your suggestion?
   
   In our production experience, a job-wide configured input constraint 
satisfies most users, together with `BATCH_FORCED` execution mode, to ensure a 
batch job can finish with limited resources.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246429765
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -686,6 +689,7 @@ void scheduleOrUpdateConsumers(ResultPartitionID 
partitionId) {
 
if 
(partition.getIntermediateResult().getResultType().isPipelined()) {
// Schedule or update receivers of this partition
+   partition.markSomePipelinedDataProduced();
 
 Review comment:
   Sure.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246429791
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
 ##
 @@ -179,6 +179,13 @@ public static ExecutionGraph buildGraph(
executionGraph.setScheduleMode(jobGraph.getScheduleMode());

executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
 
+   try {
+   executionGraph.setInputDependencyConstraint(
+
jobGraph.getSerializedExecutionConfig().deserializeValue(classLoader).getInputDependencyConstraint());
 
 Review comment:
   Good suggestion. I've move `InputDependencyConstraint ` to JobVertex.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246429640
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +730,41 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
+   if (getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY) {
+   // InputDependencyConstraint == ANY
+   return IntStream.range(0, 
inputEdges.length).anyMatch(this::isInputConsumable);
+   } else {
+   // InputDependencyConstraint == ALL
+   return IntStream.range(0, 
inputEdges.length).allMatch(this::isInputConsumable);
+   }
+   }
+
+   /**
+* An input is consumable when
+* 1. the source result is PIPELINED and one of the result partition 
has produced data.
+* 2. the source result is BLOCKING and is FINISHED(all partitions are 
FINISHED).
+*
+* @return whether the input is consumable
+*/
+   public boolean isInputConsumable(int inputNumber) {
 
 Review comment:
   A vertex `Input` is a bit different from its corresponding 
`IntermediateResult` with `POINTWISE` edge. So we need the `inputEdges` info in 
ExecutionVertex.
   
   I changed it a bit to be more concise here: An input is consumable when when 
any partition in it is consumable. (whether a partition is consumable is 
different for PIPELINED and BLOCKING results)
   
   B.T.W I'm also thinking about a later improvement that we can decide whether 
the input is consumable according to its completeness percentage. It's also a 
configuration related to `inputEdges`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2019-01-09 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r246429640
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +730,41 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
+   if (getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY) {
+   // InputDependencyConstraint == ANY
+   return IntStream.range(0, 
inputEdges.length).anyMatch(this::isInputConsumable);
+   } else {
+   // InputDependencyConstraint == ALL
+   return IntStream.range(0, 
inputEdges.length).allMatch(this::isInputConsumable);
+   }
+   }
+
+   /**
+* An input is consumable when
+* 1. the source result is PIPELINED and one of the result partition 
has produced data.
+* 2. the source result is BLOCKING and is FINISHED(all partitions are 
FINISHED).
+*
+* @return whether the input is consumable
+*/
+   public boolean isInputConsumable(int inputNumber) {
 
 Review comment:
   A vertex `Input` is a bit different from its corresponding 
`IntermediateResult` with `POINTWISE` edge. So we need the `inputEdges` info in 
ExecutionVertex.
   
   I change it a bit to be more concise here: An input is consumable when when 
any partition in it is consumable. (whether a partition is consumable is 
different for PIPELINED and BLOCKING results)
   
   B.T.W I'm also thinking about a later improvement that we can decide whether 
the input is consumable according to its completeness percentage. It's also a 
configuration related to `inputEdges`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-16 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r242017559
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +728,56 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
 
 Review comment:
   An explanation for the shortcut check is added in the comments.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-14 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241636524
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
 ##
 @@ -755,23 +775,10 @@ else if (numConsumers == 0) {
// TODO The current approach may send many 
update messages even though the consuming
// task has already been deployed with all 
necessary information. We have to check
// whether this is a problem and fix it, if it 
is.
-   CompletableFuture.supplyAsync(
-   () -> {
-   try {
-   final ExecutionGraph 
executionGraph = consumerVertex.getExecutionGraph();
-   
consumerVertex.scheduleForExecution(
-   
executionGraph.getSlotProvider(),
-   
executionGraph.isQueuedSchedulingAllowed(),
-   
LocationPreferenceConstraint.ANY, // there must be at least one known location
-   
Collections.emptySet());
-   } catch (Throwable t) {
-   consumerVertex.fail(new 
IllegalStateException("Could not schedule consumer " +
-   "vertex 
" + consumerVertex, t));
-   }
-
-   return null;
-   },
-   executor);
+   if 
(consumerVertex.checkInputDependencyConstraints()) {
 
 Review comment:
   From my understanding, the TODO comment is related to the "`consumerState == 
CREATED`" section in `scheduleOrUpdateConsumers`, which invokes 
`cachePartitionInfo` first and then schedules the vertex. The 
cachePartitionInfo action is needed to avoid deployment race, at the cost of 
redundant partition infos to update to task, which is the concern as described 
in the TODO comment.
   
   So far the redundant partition it's not a big problem. But I think we can 
optimize it later. One possible solution in my mind is to remove known 
partition infos from the cache when creating InputChannelDeploymentDescriptor.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241657101
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +728,56 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
 
 Review comment:
   We can skip the input constraint check for `InputDependencyConstraint.ANY` 
because when invoking `scheduleOrUpdateConsumers`, at least one of the consumer 
vertex's inputs is consumable. This assumption may not be right if 
`checkInputDependencyConstraints` is invoked from other entries.
   That's why I think it's better to add the shortcut in 
`scheduleOrUpdateConsumers`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241652922
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.runtime.executiongraph;
+
+import org.apache.flink.api.common.InputDependencyConstraint;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import 
org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.isInExecutionState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitForAllExecutionsPredicate;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilJobStatus;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the inputs constraint for {@link ExecutionVertex}.
+ */
+public class ExecutionVertexInputConstraintTest extends TestLogger {
+
+   @Test
+   public void testInputConsumable() throws Exception {
+   JobVertex v1 = new JobVertex("vertex1");
+   JobVertex v2 = new JobVertex("vertex2");
+   JobVertex v3 = new JobVertex("vertex3");
+   v1.setParallelism(2);
+   v2.setParallelism(2);
+   v3.setParallelism(2);
+   v1.setInvokableClass(AbstractInvokable.class);
+   v2.setInvokableClass(AbstractInvokable.class);
+   v3.setInvokableClass(AbstractInvokable.class);
+   v3.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE, 
ResultPartitionType.PIPELINED);
+   v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, 
ResultPartitionType.BLOCKING);
+   List ordered = Arrays.asList(v1, v2, v3);
+   ExecutionGraph eg = createExecutionGraph(ordered);
+
+   ExecutionVertex ev11 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[0];
+   ExecutionVertex ev12 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[1];
+   ExecutionVertex ev21 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[0];
+   ExecutionVertex ev22 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[1];
+   ExecutionVertex ev31 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[0];
+   ExecutionVertex ev32 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[1];
+
+   eg.scheduleForExecution();
 
 Review comment:
   I will move the shared logics to a common method. As the ExecutionGraph 
states is changing so I'd like each test maintain its own EG.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241652553
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.runtime.executiongraph;
+
+import org.apache.flink.api.common.InputDependencyConstraint;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import 
org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.isInExecutionState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitForAllExecutionsPredicate;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilJobStatus;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the inputs constraint for {@link ExecutionVertex}.
+ */
+public class ExecutionVertexInputConstraintTest extends TestLogger {
+
+   @Test
+   public void testInputConsumable() throws Exception {
+   JobVertex v1 = new JobVertex("vertex1");
+   JobVertex v2 = new JobVertex("vertex2");
+   JobVertex v3 = new JobVertex("vertex3");
+   v1.setParallelism(2);
+   v2.setParallelism(2);
+   v3.setParallelism(2);
+   v1.setInvokableClass(AbstractInvokable.class);
+   v2.setInvokableClass(AbstractInvokable.class);
+   v3.setInvokableClass(AbstractInvokable.class);
+   v3.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE, 
ResultPartitionType.PIPELINED);
+   v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, 
ResultPartitionType.BLOCKING);
+   List ordered = Arrays.asList(v1, v2, v3);
+   ExecutionGraph eg = createExecutionGraph(ordered);
+
+   ExecutionVertex ev11 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[0];
+   ExecutionVertex ev12 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[1];
+   ExecutionVertex ev21 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[0];
+   ExecutionVertex ev22 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[1];
+   ExecutionVertex ev31 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[0];
+   ExecutionVertex ev32 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[1];
+
+   eg.scheduleForExecution();
+
+   // Inputs not consumable on init
+   assertFalse(ev31.isInputConsumable(0));
+   assertFalse(ev31.isInputConsumable(1));
+
+   // One pipelined input consumable on data produced
+   IntermediateResultPartition partition11 = 
ev11.getProducedPartitions().values().iterator().next();
+   ev11.scheduleOrUpdateConsumers(new 
ResultPartitionID(partition11.getPartitionId(),
+   ev11.getCurrentExecutionAttempt().getAttemptId()));
+

[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241652478
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java
 ##
 @@ -0,0 +1,98 @@
+/*
+ * 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.runtime.executiongraph;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link IntermediateResultPartition}.
+ */
+public class IntermediateResultPartitionTest extends TestLogger {
+
+   @Test
+   public void testPipelinedPartitionConsumable() throws Exception {
+   ExecutionJobVertex jobVertex = getExecutionVertex(new 
JobVertexID(), new DirectScheduledExecutorService());
+   IntermediateResult result =
+   new IntermediateResult(new IntermediateDataSetID(), 
jobVertex, 2, ResultPartitionType.PIPELINED);
+   ExecutionVertex vertex1 =
+   new ExecutionVertex(jobVertex, 0, new 
IntermediateResult[]{result}, Time.minutes(1));
+   ExecutionVertex vertex2 =
+   new ExecutionVertex(jobVertex, 1, new 
IntermediateResult[]{result}, Time.minutes(1));
+
+   IntermediateResultPartition partition1 = 
result.getPartitions()[0];
+   IntermediateResultPartition partition2 = 
result.getPartitions()[1];
 
 Review comment:
   Agree. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241652506
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.runtime.executiongraph;
+
+import org.apache.flink.api.common.InputDependencyConstraint;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import 
org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.isInExecutionState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitForAllExecutionsPredicate;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilJobStatus;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the inputs constraint for {@link ExecutionVertex}.
+ */
+public class ExecutionVertexInputConstraintTest extends TestLogger {
+
+   @Test
+   public void testInputConsumable() throws Exception {
+   JobVertex v1 = new JobVertex("vertex1");
+   JobVertex v2 = new JobVertex("vertex2");
+   JobVertex v3 = new JobVertex("vertex3");
+   v1.setParallelism(2);
+   v2.setParallelism(2);
+   v3.setParallelism(2);
+   v1.setInvokableClass(AbstractInvokable.class);
+   v2.setInvokableClass(AbstractInvokable.class);
+   v3.setInvokableClass(AbstractInvokable.class);
+   v3.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE, 
ResultPartitionType.PIPELINED);
+   v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, 
ResultPartitionType.BLOCKING);
+   List ordered = Arrays.asList(v1, v2, v3);
+   ExecutionGraph eg = createExecutionGraph(ordered);
+
+   ExecutionVertex ev11 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[0];
+   ExecutionVertex ev12 = 
eg.getJobVertex(v1.getID()).getTaskVertices()[1];
+   ExecutionVertex ev21 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[0];
+   ExecutionVertex ev22 = 
eg.getJobVertex(v2.getID()).getTaskVertices()[1];
+   ExecutionVertex ev31 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[0];
+   ExecutionVertex ev32 = 
eg.getJobVertex(v3.getID()).getTaskVertices()[1];
+
+   eg.scheduleForExecution();
+
+   // Inputs not consumable on init
+   assertFalse(ev31.isInputConsumable(0));
+   assertFalse(ev31.isInputConsumable(1));
+
+   // One pipelined input consumable on data produced
+   IntermediateResultPartition partition11 = 
ev11.getProducedPartitions().values().iterator().next();
+   ev11.scheduleOrUpdateConsumers(new 
ResultPartitionID(partition11.getPartitionId(),
+   ev11.getCurrentExecutionAttempt().getAttemptId()));
+

[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241636524
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
 ##
 @@ -755,23 +775,10 @@ else if (numConsumers == 0) {
// TODO The current approach may send many 
update messages even though the consuming
// task has already been deployed with all 
necessary information. We have to check
// whether this is a problem and fix it, if it 
is.
-   CompletableFuture.supplyAsync(
-   () -> {
-   try {
-   final ExecutionGraph 
executionGraph = consumerVertex.getExecutionGraph();
-   
consumerVertex.scheduleForExecution(
-   
executionGraph.getSlotProvider(),
-   
executionGraph.isQueuedSchedulingAllowed(),
-   
LocationPreferenceConstraint.ANY, // there must be at least one known location
-   
Collections.emptySet());
-   } catch (Throwable t) {
-   consumerVertex.fail(new 
IllegalStateException("Could not schedule consumer " +
-   "vertex 
" + consumerVertex, t));
-   }
-
-   return null;
-   },
-   executor);
+   if 
(consumerVertex.checkInputDependencyConstraints()) {
 
 Review comment:
   From my understanding, the TODO comment is related to the "consumerState == 
CREATED" section in scheduleOrUpdateConsumers, which invokes cachePartitionInfo 
first and then schedules the vertex. The cachePartitionInfo action is needed to 
avoid deployment race, at the cost of redundant partition infos to update to 
task, which is the concern as described in the TODO comment.
   
   So far the redundant partition it's not a big problem. But I think we can 
optimize it later. One possible solution in my mind is to remove known 
partition infos from the cache when creating InputChannelDeploymentDescriptor.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-13 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r241636524
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
 ##
 @@ -755,23 +775,10 @@ else if (numConsumers == 0) {
// TODO The current approach may send many 
update messages even though the consuming
// task has already been deployed with all 
necessary information. We have to check
// whether this is a problem and fix it, if it 
is.
-   CompletableFuture.supplyAsync(
-   () -> {
-   try {
-   final ExecutionGraph 
executionGraph = consumerVertex.getExecutionGraph();
-   
consumerVertex.scheduleForExecution(
-   
executionGraph.getSlotProvider(),
-   
executionGraph.isQueuedSchedulingAllowed(),
-   
LocationPreferenceConstraint.ANY, // there must be at least one known location
-   
Collections.emptySet());
-   } catch (Throwable t) {
-   consumerVertex.fail(new 
IllegalStateException("Could not schedule consumer " +
-   "vertex 
" + consumerVertex, t));
-   }
-
-   return null;
-   },
-   executor);
+   if 
(consumerVertex.checkInputDependencyConstraints()) {
 
 Review comment:
   From my understanding, the TODO comment is related to the "consumerState == 
CREATED" section in scheduleOrUpdateConsumers, which invokes cachePartitionInfo 
first and then schedules the vertex. The cachePartitionInfo action is used to 
avoid deployment race at the cost of redundant partition infos to update to 
task, which is the concern as described in the TODO comment.
   
   So far the redundant partition it's not a big problem. But I think we can 
optimize it later. One possible solution in my mind is to remove known 
partition infos from the cache when creating InputChannelDeploymentDescriptor.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-11 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240655589
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##
 @@ -726,6 +728,56 @@ void sendPartitionInfos() {
}
}
 
+   /**
+* Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+*
+* @return whether the input constraint is satisfied
+*/
+   public boolean checkInputDependencyConstraints() {
 
 Review comment:
   Thanks for the suggestion. I made it more concise, but a bit different from 
your sample code:
   For PIPELINED input consumable check, only the partitions related with the 
input edges are checked. 
   
   I'm not sure why you think the behavior is changed? 
   I'd agree if you have concerns that this check could be heavy(O(N) 
complexity)) for large parallelism vertex. So I'm considering whether to add a 
short cut check for `InputDependencyConstraint.ANY` in 
`Execution.scheduleOrUpdateConsumers`, i.e. change `if 
(consumerVertex.checkInputDependencyConstraints()) ` to `if 
(getVertex().getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY || 
consumerVertex.checkInputDependencyConstraints())`. It's not elegant but can be 
as efficient as current scheduling.
   
   The input state is not used in EAGER scheduling. So the 
InputDependencyConstraint is not a matter in EAGER mode. It is used in 
scheduleOrUpdateConsumers, which is only invoked in LAZY_FROM_SOURCES 
scheduling.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-11 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240629621
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
 ##
 @@ -36,6 +36,11 @@
 
private List> consumers;
 
+   /**
+* Whether this partition has data produced. For pipelined result only.
 
 Review comment:
   Agree.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-11 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240629262
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
 ##
 @@ -747,31 +747,34 @@ else if (numConsumers == 0) {

consumerVertex.cachePartitionInfo(PartialInputChannelDeploymentDescriptor.fromEdge(
partition, partitionExecution));
 
-   // When deploying a consuming task, its task 
deployment descriptor will contain all
-   // deployment information available at the 
respective time. It is possible that some
-   // of the partitions to be consumed have not 
been created yet. These are updated
-   // runtime via the update messages.
-   //
-   // TODO The current approach may send many 
update messages even though the consuming
-   // task has already been deployed with all 
necessary information. We have to check
-   // whether this is a problem and fix it, if it 
is.
-   CompletableFuture.supplyAsync(
-   () -> {
-   try {
-   final ExecutionGraph 
executionGraph = consumerVertex.getExecutionGraph();
-   
consumerVertex.scheduleForExecution(
-   
executionGraph.getSlotProvider(),
-   
executionGraph.isQueuedSchedulingAllowed(),
-   
LocationPreferenceConstraint.ANY, // there must be at least one known location
-   
Collections.emptySet());
-   } catch (Throwable t) {
-   consumerVertex.fail(new 
IllegalStateException("Could not schedule consumer " +
+   // Schedule the consumer vertex if its inputs 
constraint is satisfied, otherwise postpone the scheduling
+   if 
(consumerVertex.checkInputDependencyConstraints()) {
+   // When deploying a consuming task, its 
task deployment descriptor will contain all
+   // deployment information available at 
the respective time. It is possible that some
+   // of the partitions to be consumed 
have not been created yet. These are updated
+   // runtime via the update messages.
+   //
+   // TODO The current approach may send 
many update messages even though the consuming
+   // task has already been deployed with 
all necessary information. We have to check
+   // whether this is a problem and fix 
it, if it is.
+   CompletableFuture.supplyAsync(
 
 Review comment:
   Sure.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-11 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240629143
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
 ##
 @@ -36,6 +36,11 @@
 
private List> consumers;
 
+   /**
+* Whether this partition has data produced. For pipelined result only.
+*/
+   private boolean dataProduced = false;
 
 Review comment:
   That's fine.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use InputDependencyConstraint to avoid resource dead…

2018-12-11 Thread GitBox
zhuzhurk commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240628670
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java
 ##
 @@ -192,6 +193,7 @@ private static ExecutionVertex 
mockExecutionVertex(ExecutionState state, Resourc
 
private static IntermediateResultPartition 
mockPartition(ExecutionVertex producer) {
IntermediateResultPartition partition = 
mock(IntermediateResultPartition.class);
+   
when(partition.getResultType()).thenReturn(ResultPartitionType.PIPELINED);
 
 Review comment:
   Yes. It is needed to avoid NPE in the tests.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services