You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by uc...@apache.org on 2016/11/11 08:45:01 UTC
[2/3] flink git commit: [FLINK-5040] [jobmanager] Set correct input
channel types with eager scheduling
[FLINK-5040] [jobmanager] Set correct input channel types with eager scheduling
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2742d5c1
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2742d5c1
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2742d5c1
Branch: refs/heads/master
Commit: 2742d5c1761ca02d871333e91a8ecbc6d0a52f6c
Parents: 0d2e8b2
Author: Ufuk Celebi <uc...@apache.org>
Authored: Wed Nov 9 18:25:06 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Nov 10 23:06:55 2016 +0100
----------------------------------------------------------------------
.../ResultPartitionDeploymentDescriptor.java | 17 +-
.../runtime/executiongraph/ExecutionVertex.java | 21 +-
.../runtime/io/network/PartitionState.java | 18 +-
.../io/network/partition/ResultPartition.java | 8 +-
.../flink/runtime/jobgraph/ScheduleMode.java | 10 +-
.../apache/flink/runtime/taskmanager/Task.java | 10 +-
.../flink/runtime/jobmanager/JobManager.scala | 2 +-
.../InputChannelDeploymentDescriptorTest.java | 206 +++++++++++++++++++
...ResultPartitionDeploymentDescriptorTest.java | 6 +-
.../ExecutionVertexDeploymentTest.java | 106 ++++++----
.../network/partition/ResultPartitionTest.java | 92 +++++++++
.../consumer/LocalInputChannelTest.java | 3 +-
.../runtime/jobgraph/ScheduleModeTest.java | 36 ++++
.../runtime/taskmanager/TaskManagerTest.java | 19 +-
.../flink/runtime/taskmanager/TaskTest.java | 5 +-
15 files changed, 491 insertions(+), 68 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
index 2881dde..2ecde80 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java
@@ -47,12 +47,16 @@ public class ResultPartitionDeploymentDescriptor implements Serializable {
/** The number of subpartitions. */
private final int numberOfSubpartitions;
+
+ /** Flag whether the result partition should send scheduleOrUpdateConsumer messages. */
+ private final boolean lazyScheduling;
public ResultPartitionDeploymentDescriptor(
IntermediateDataSetID resultId,
IntermediateResultPartitionID partitionId,
ResultPartitionType partitionType,
- int numberOfSubpartitions) {
+ int numberOfSubpartitions,
+ boolean lazyScheduling) {
this.resultId = checkNotNull(resultId);
this.partitionId = checkNotNull(partitionId);
@@ -60,6 +64,7 @@ public class ResultPartitionDeploymentDescriptor implements Serializable {
checkArgument(numberOfSubpartitions >= 1);
this.numberOfSubpartitions = numberOfSubpartitions;
+ this.lazyScheduling = lazyScheduling;
}
public IntermediateDataSetID getResultId() {
@@ -78,6 +83,10 @@ public class ResultPartitionDeploymentDescriptor implements Serializable {
return numberOfSubpartitions;
}
+ public boolean allowLazyScheduling() {
+ return lazyScheduling;
+ }
+
@Override
public String toString() {
return String.format("ResultPartitionDeploymentDescriptor [result id: %s, "
@@ -87,7 +96,7 @@ public class ResultPartitionDeploymentDescriptor implements Serializable {
// ------------------------------------------------------------------------
- public static ResultPartitionDeploymentDescriptor from(IntermediateResultPartition partition) {
+ public static ResultPartitionDeploymentDescriptor from(IntermediateResultPartition partition, boolean lazyScheduling) {
final IntermediateDataSetID resultId = partition.getIntermediateResult().getId();
final IntermediateResultPartitionID partitionId = partition.getPartitionId();
@@ -102,13 +111,13 @@ public class ResultPartitionDeploymentDescriptor implements Serializable {
if (!partition.getConsumers().isEmpty() && !partition.getConsumers().get(0).isEmpty()) {
if (partition.getConsumers().size() > 1) {
- new IllegalStateException("Currently, only a single consumer group per partition is supported.");
+ throw new IllegalStateException("Currently, only a single consumer group per partition is supported.");
}
numberOfSubpartitions = partition.getConsumers().get(0).size();
}
return new ResultPartitionDeploymentDescriptor(
- resultId, partitionId, partitionType, numberOfSubpartitions);
+ resultId, partitionId, partitionType, numberOfSubpartitions, lazyScheduling);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index e7f000c..01e8660 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -568,21 +568,24 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
ExecutionAttemptID executionId,
SimpleSlot targetSlot,
TaskStateHandles taskStateHandles,
- int attemptNumber) {
-
+ int attemptNumber) throws ExecutionGraphException {
+
// Produced intermediate results
- List<ResultPartitionDeploymentDescriptor> producedPartitions = new ArrayList<ResultPartitionDeploymentDescriptor>(resultPartitions.size());
+ List<ResultPartitionDeploymentDescriptor> producedPartitions = new ArrayList<>(resultPartitions.size());
+
+ // Consumed intermediate results
+ List<InputGateDeploymentDescriptor> consumedPartitions = new ArrayList<>(inputEdges.length);
+
+ boolean lazyScheduling = getExecutionGraph().getScheduleMode().allowLazyDeployment();
for (IntermediateResultPartition partition : resultPartitions.values()) {
- producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition));
+ producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition, lazyScheduling));
}
-
- // Consumed intermediate results
- List<InputGateDeploymentDescriptor> consumedPartitions = new ArrayList<InputGateDeploymentDescriptor>();
-
+
+
for (ExecutionEdge[] edges : inputEdges) {
InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor
- .fromEdges(edges, targetSlot);
+ .fromEdges(edges, targetSlot, lazyScheduling);
// If the produced partition has multiple consumers registered, we
// need to request the one matching our sub task index.
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionState.java
index 083412b..59357fc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionState.java
@@ -23,18 +23,25 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.util.Preconditions;
+import javax.annotation.Nullable;
+import java.io.Serializable;
+
/**
* Contains information about the state of a result partition.
*/
-public class PartitionState {
+public class PartitionState implements Serializable {
+
+ private static final long serialVersionUID = -4693651272083825031L;
+
private final IntermediateDataSetID intermediateDataSetID;
private final IntermediateResultPartitionID intermediateResultPartitionID;
private final ExecutionState executionState;
public PartitionState(
- IntermediateDataSetID intermediateDataSetID,
- IntermediateResultPartitionID intermediateResultPartitionID,
- ExecutionState executionState) {
+ IntermediateDataSetID intermediateDataSetID,
+ IntermediateResultPartitionID intermediateResultPartitionID,
+ @Nullable ExecutionState executionState) {
+
this.intermediateDataSetID = Preconditions.checkNotNull(intermediateDataSetID);
this.intermediateResultPartitionID = Preconditions.checkNotNull(intermediateResultPartitionID);
this.executionState = executionState;
@@ -48,6 +55,9 @@ public class PartitionState {
return intermediateResultPartitionID;
}
+ /**
+ * Returns the execution state of the partition producer or <code>null</code> if it is not available.
+ */
public ExecutionState getExecutionState() {
return executionState;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
index 034b27a..834318c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
@@ -96,6 +96,8 @@ public class ResultPartition implements BufferPoolOwner {
private final ResultPartitionConsumableNotifier partitionConsumableNotifier;
+ private final boolean sendScheduleOrUpdateConsumersMessage;
+
// - Runtime state --------------------------------------------------------
private final AtomicBoolean isReleased = new AtomicBoolean();
@@ -133,7 +135,8 @@ public class ResultPartition implements BufferPoolOwner {
ResultPartitionManager partitionManager,
ResultPartitionConsumableNotifier partitionConsumableNotifier,
IOManager ioManager,
- IOMode defaultIoMode) {
+ IOMode defaultIoMode,
+ boolean sendScheduleOrUpdateConsumersMessage) {
this.owningTaskName = checkNotNull(owningTaskName);
this.taskActions = checkNotNull(taskActions);
@@ -143,6 +146,7 @@ public class ResultPartition implements BufferPoolOwner {
this.subpartitions = new ResultSubpartition[numberOfSubpartitions];
this.partitionManager = checkNotNull(partitionManager);
this.partitionConsumableNotifier = checkNotNull(partitionConsumableNotifier);
+ this.sendScheduleOrUpdateConsumersMessage = sendScheduleOrUpdateConsumersMessage;
// Create the subpartitions.
switch (partitionType) {
@@ -437,7 +441,7 @@ public class ResultPartition implements BufferPoolOwner {
* Notifies pipelined consumers of this result partition once.
*/
private void notifyPipelinedConsumers() {
- if (partitionType.isPipelined() && !hasNotifiedPipelinedConsumers) {
+ if (sendScheduleOrUpdateConsumersMessage && !hasNotifiedPipelinedConsumers && partitionType.isPipelined()) {
partitionConsumableNotifier.notifyPartitionConsumable(jobId, partitionId, taskActions);
hasNotifiedPipelinedConsumers = true;
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ScheduleMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ScheduleMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ScheduleMode.java
index 9405067..6a98e46 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ScheduleMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ScheduleMode.java
@@ -27,5 +27,13 @@ public enum ScheduleMode {
LAZY_FROM_SOURCES,
/** Schedules all tasks immediately. */
- EAGER
+ EAGER;
+
+ /**
+ * Returns whether we are allowed to deploy consumers lazily.
+ */
+ public boolean allowLazyDeployment() {
+ return this == LAZY_FROM_SOURCES;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 6907606..4f3dd54 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -345,7 +345,8 @@ public class Task implements Runnable, TaskActions {
networkEnvironment.getResultPartitionManager(),
resultPartitionConsumableNotifier,
ioManager,
- networkEnvironment.getDefaultIOMode());
+ networkEnvironment.getDefaultIOMode(),
+ desc.allowLazyScheduling());
writers[counter] = new ResultPartitionWriter(producedPartitions[counter]);
@@ -568,6 +569,7 @@ public class Task implements Runnable, TaskActions {
// ----------------------------------------------------------------
LOG.info("Registering task at network: " + this);
+
network.registerTask(this);
// next, kick off the background copying of files for the distributed cache
@@ -1135,7 +1137,11 @@ public class Task implements Runnable, TaskActions {
final SingleInputGate inputGate = inputGatesById.get(resultId);
if (inputGate != null) {
- if (partitionState == ExecutionState.RUNNING) {
+ if (partitionState == ExecutionState.RUNNING ||
+ partitionState == ExecutionState.FINISHED ||
+ partitionState == ExecutionState.SCHEDULED ||
+ partitionState == ExecutionState.DEPLOYING) {
+
// Retrigger the partition request
inputGate.retriggerPartitionRequest(partitionId);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 9af5355..b2e1002 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -948,7 +948,7 @@ class JobManager(
if (execution != null) execution.getState else null
case None =>
// Nothing to do. This is not an error, because the request is received when a sending
- // task fails during a remote partition request.
+ // task fails or is not yet available during a remote partition request.
log.debug(s"Cannot find execution graph for job $jobId.")
null
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java
new file mode 100644
index 0000000..e9e8901
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptorTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.deployment;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphException;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.junit.Test;
+
+import java.net.InetAddress;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class InputChannelDeploymentDescriptorTest {
+
+ /**
+ * Tests the deployment descriptors for local, remote, and unknown partition
+ * locations (with lazy deployment allowed and all execution states for the
+ * producers).
+ */
+ @Test
+ public void testMixedLocalRemoteUnknownDeployment() throws Exception {
+ boolean allowLazyDeployment = true;
+
+ ResourceID consumerResourceId = ResourceID.generate();
+ ExecutionVertex consumer = mock(ExecutionVertex.class);
+ SimpleSlot consumerSlot = mockSlot(consumerResourceId);
+
+ // Local and remote channel are only allowed for certain execution
+ // states.
+ for (ExecutionState state : ExecutionState.values()) {
+ // Local partition
+ ExecutionVertex localProducer = mockExecutionVertex(state, consumerResourceId);
+ IntermediateResultPartition localPartition = mockPartition(localProducer);
+ ResultPartitionID localPartitionId = new ResultPartitionID(localPartition.getPartitionId(), localProducer.getCurrentExecutionAttempt().getAttemptId());
+ ExecutionEdge localEdge = new ExecutionEdge(localPartition, consumer, 0);
+
+ // Remote partition
+ ExecutionVertex remoteProducer = mockExecutionVertex(state, ResourceID.generate()); // new resource ID
+ IntermediateResultPartition remotePartition = mockPartition(remoteProducer);
+ ResultPartitionID remotePartitionId = new ResultPartitionID(remotePartition.getPartitionId(), remoteProducer.getCurrentExecutionAttempt().getAttemptId());
+ ConnectionID remoteConnectionId = new ConnectionID(remoteProducer.getCurrentAssignedResource().getTaskManagerLocation(), 0);
+ ExecutionEdge remoteEdge = new ExecutionEdge(remotePartition, consumer, 1);
+
+ // Unknown partition
+ ExecutionVertex unknownProducer = mockExecutionVertex(state, null); // no assigned resource
+ IntermediateResultPartition unknownPartition = mockPartition(unknownProducer);
+ ResultPartitionID unknownPartitionId = new ResultPartitionID(unknownPartition.getPartitionId(), unknownProducer.getCurrentExecutionAttempt().getAttemptId());
+ ExecutionEdge unknownEdge = new ExecutionEdge(unknownPartition, consumer, 2);
+
+ InputChannelDeploymentDescriptor[] desc = InputChannelDeploymentDescriptor.fromEdges(
+ new ExecutionEdge[]{localEdge, remoteEdge, unknownEdge},
+ consumerSlot,
+ allowLazyDeployment);
+
+ assertEquals(3, desc.length);
+
+ // These states are allowed
+ if (state == ExecutionState.RUNNING || state == ExecutionState.FINISHED ||
+ state == ExecutionState.SCHEDULED || state == ExecutionState.DEPLOYING) {
+
+ // Create local or remote channels
+ assertEquals(localPartitionId, desc[0].getConsumedPartitionId());
+ assertTrue(desc[0].getConsumedPartitionLocation().isLocal());
+ assertNull(desc[0].getConsumedPartitionLocation().getConnectionId());
+
+ assertEquals(remotePartitionId, desc[1].getConsumedPartitionId());
+ assertTrue(desc[1].getConsumedPartitionLocation().isRemote());
+ assertEquals(remoteConnectionId, desc[1].getConsumedPartitionLocation().getConnectionId());
+ } else {
+ // Unknown (lazy deployment allowed)
+ assertEquals(localPartitionId, desc[0].getConsumedPartitionId());
+ assertTrue(desc[0].getConsumedPartitionLocation().isUnknown());
+ assertNull(desc[0].getConsumedPartitionLocation().getConnectionId());
+
+ assertEquals(remotePartitionId, desc[1].getConsumedPartitionId());
+ assertTrue(desc[1].getConsumedPartitionLocation().isUnknown());
+ assertNull(desc[1].getConsumedPartitionLocation().getConnectionId());
+ }
+
+ assertEquals(unknownPartitionId, desc[2].getConsumedPartitionId());
+ assertTrue(desc[2].getConsumedPartitionLocation().isUnknown());
+ assertNull(desc[2].getConsumedPartitionLocation().getConnectionId());
+ }
+ }
+
+ @Test
+ public void testUnknownChannelWithoutLazyDeploymentThrows() throws Exception {
+ ResourceID consumerResourceId = ResourceID.generate();
+ ExecutionVertex consumer = mock(ExecutionVertex.class);
+ SimpleSlot consumerSlot = mockSlot(consumerResourceId);
+
+
+ // Unknown partition
+ ExecutionVertex unknownProducer = mockExecutionVertex(ExecutionState.CREATED, null); // no assigned resource
+ IntermediateResultPartition unknownPartition = mockPartition(unknownProducer);
+ ResultPartitionID unknownPartitionId = new ResultPartitionID(unknownPartition.getPartitionId(), unknownProducer.getCurrentExecutionAttempt().getAttemptId());
+ ExecutionEdge unknownEdge = new ExecutionEdge(unknownPartition, consumer, 2);
+
+ // This should work if lazy deployment is allowed
+ boolean allowLazyDeployment = true;
+
+ InputChannelDeploymentDescriptor[] desc = InputChannelDeploymentDescriptor.fromEdges(
+ new ExecutionEdge[]{unknownEdge},
+ consumerSlot,
+ allowLazyDeployment);
+
+ assertEquals(1, desc.length);
+
+ assertEquals(unknownPartitionId, desc[0].getConsumedPartitionId());
+ assertTrue(desc[0].getConsumedPartitionLocation().isUnknown());
+ assertNull(desc[0].getConsumedPartitionLocation().getConnectionId());
+
+
+ try {
+ // Fail if lazy deployment is *not* allowed
+ allowLazyDeployment = false;
+
+ InputChannelDeploymentDescriptor.fromEdges(
+ new ExecutionEdge[]{unknownEdge},
+ consumerSlot,
+ allowLazyDeployment);
+
+ fail("Did not throw expected ExecutionGraphException");
+ } catch (ExecutionGraphException ignored) {
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ private static SimpleSlot mockSlot(ResourceID resourceId) {
+ SimpleSlot slot = mock(SimpleSlot.class);
+ when(slot.getTaskManagerLocation()).thenReturn(new TaskManagerLocation(resourceId, InetAddress.getLoopbackAddress(), 5000));
+ when(slot.getTaskManagerID()).thenReturn(resourceId);
+
+ return slot;
+ }
+
+ private static ExecutionVertex mockExecutionVertex(ExecutionState state, ResourceID resourceId) {
+ ExecutionVertex vertex = mock(ExecutionVertex.class);
+
+ Execution exec = mock(Execution.class);
+ when(exec.getState()).thenReturn(state);
+ when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID());
+
+ if (resourceId != null) {
+ SimpleSlot slot = mockSlot(resourceId);
+ when(exec.getAssignedResource()).thenReturn(slot);
+ when(vertex.getCurrentAssignedResource()).thenReturn(slot);
+ } else {
+ when(exec.getAssignedResource()).thenReturn(null); // no resource
+ when(vertex.getCurrentAssignedResource()).thenReturn(null);
+ }
+
+ when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
+
+ return vertex;
+ }
+
+ private static IntermediateResultPartition mockPartition(ExecutionVertex producer) {
+ IntermediateResultPartition partition = mock(IntermediateResultPartition.class);
+ when(partition.isConsumable()).thenReturn(true);
+
+ IntermediateResult result = mock(IntermediateResult.class);
+ when(result.getConnectionIndex()).thenReturn(0);
+
+ when(partition.getIntermediateResult()).thenReturn(result);
+ when(partition.getPartitionId()).thenReturn(new IntermediateResultPartitionID());
+
+ when(partition.getProducer()).thenReturn(producer);
+
+ return partition;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java
index 4b1e546..4223b49 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
public class ResultPartitionDeploymentDescriptorTest {
@@ -38,14 +39,14 @@ public class ResultPartitionDeploymentDescriptorTest {
IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
ResultPartitionType partitionType = ResultPartitionType.PIPELINED;
int numberOfSubpartitions = 24;
- boolean eagerlyDeployConsumers = true;
ResultPartitionDeploymentDescriptor orig =
new ResultPartitionDeploymentDescriptor(
resultId,
partitionId,
partitionType,
- numberOfSubpartitions);
+ numberOfSubpartitions,
+ true);
ResultPartitionDeploymentDescriptor copy =
CommonTestUtils.createCopySerializable(orig);
@@ -54,5 +55,6 @@ public class ResultPartitionDeploymentDescriptorTest {
assertEquals(partitionId, copy.getPartitionId());
assertEquals(partitionType, copy.getPartitionType());
assertEquals(numberOfSubpartitions, copy.getNumberOfSubpartitions());
+ assertTrue(copy.allowLazyScheduling());
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
index 54aeff9..8bc39a7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
@@ -18,20 +18,37 @@
package org.apache.flink.runtime.executiongraph;
-import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*;
-
-import static org.junit.Assert.*;
-
+import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.Slot;
+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.jobgraph.ScheduleMode;
import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway;
import org.apache.flink.runtime.testingUtils.TestingUtils;
-
import org.junit.Test;
+import java.util.Collection;
+
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.ERROR_MESSAGE;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleFailingActorGateway;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
public class ExecutionVertexDeploymentTest {
@Test
@@ -48,7 +65,7 @@ public class ExecutionVertexDeploymentTest {
final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId());
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
vertex.deployToSlot(slot);
@@ -58,8 +75,7 @@ public class ExecutionVertexDeploymentTest {
try {
vertex.deployToSlot(slot);
fail("Scheduled from wrong state");
- }
- catch (IllegalStateException e) {
+ } catch (IllegalStateException e) {
// as expected
}
@@ -67,8 +83,7 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -82,12 +97,12 @@ public class ExecutionVertexDeploymentTest {
final ExecutionJobVertex ejv = getExecutionVertex(jid, TestingUtils.directExecutionContext());
final Instance instance = getInstance(
- new ActorTaskManagerGateway(
- new SimpleActorGateway(TestingUtils.directExecutionContext())));
+ new ActorTaskManagerGateway(
+ new SimpleActorGateway(TestingUtils.directExecutionContext())));
final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId());
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
@@ -99,8 +114,7 @@ public class ExecutionVertexDeploymentTest {
try {
vertex.deployToSlot(slot);
fail("Scheduled from wrong state");
- }
- catch (IllegalStateException e) {
+ } catch (IllegalStateException e) {
// as expected
}
@@ -109,8 +123,7 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.RUNNING) == 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -123,7 +136,7 @@ public class ExecutionVertexDeploymentTest {
final ExecutionJobVertex ejv = getExecutionVertex(jid);
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
final Instance instance = getInstance(
new ActorTaskManagerGateway(
@@ -138,8 +151,7 @@ public class ExecutionVertexDeploymentTest {
try {
vertex.deployToSlot(slot);
fail("Scheduled from wrong state");
- }
- catch (IllegalStateException e) {
+ } catch (IllegalStateException e) {
// as expected
}
@@ -149,16 +161,14 @@ public class ExecutionVertexDeploymentTest {
try {
vertex.deployToSlot(slot);
fail("Scheduled from wrong state");
- }
- catch (IllegalStateException e) {
+ } catch (IllegalStateException e) {
// as expected
}
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.RUNNING) == 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -171,7 +181,7 @@ public class ExecutionVertexDeploymentTest {
final ExecutionJobVertex ejv = getExecutionVertex(jid, TestingUtils.directExecutionContext());
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
final Instance instance = getInstance(
new ActorTaskManagerGateway(
@@ -189,8 +199,7 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -202,7 +211,7 @@ public class ExecutionVertexDeploymentTest {
final JobVertexID jid = new JobVertexID();
final ExecutionJobVertex ejv = getExecutionVertex(jid);
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
final Instance instance = getInstance(
new ActorTaskManagerGateway(
@@ -229,8 +238,7 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -247,7 +255,7 @@ public class ExecutionVertexDeploymentTest {
final ExecutionJobVertex ejv = getExecutionVertex(jid, ec);
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
final Instance instance = getInstance(
new ActorTaskManagerGateway(
@@ -270,8 +278,7 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
@@ -288,7 +295,7 @@ public class ExecutionVertexDeploymentTest {
final ExecutionJobVertex ejv = getExecutionVertex(jid, context);
final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0],
- AkkaUtils.getDefaultTimeout());
+ AkkaUtils.getDefaultTimeout());
final ExecutionAttemptID eid = vertex.getCurrentExecutionAttempt().getAttemptId();
@@ -334,10 +341,37 @@ public class ExecutionVertexDeploymentTest {
assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
assertTrue(queue.isEmpty());
- }
- catch (Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
}
+
+ /**
+ * Tests that the lazy scheduling flag is correctly forwarded to the produced partition descriptors.
+ */
+ @Test
+ public void testTddProducedPartitionsLazyScheduling() throws Exception {
+ TestingUtils.QueuedActionExecutionContext context = TestingUtils.queuedActionExecutionContext();
+ ExecutionJobVertex jobVertex = getExecutionVertex(new JobVertexID(), context);
+ IntermediateResult result = new IntermediateResult(new IntermediateDataSetID(), jobVertex, 4, ResultPartitionType.PIPELINED);
+ ExecutionVertex vertex = new ExecutionVertex(jobVertex, 0, new IntermediateResult[]{result}, Time.minutes(1));
+
+ Slot root = mock(Slot.class);
+ when(root.getSlotNumber()).thenReturn(1);
+ SimpleSlot slot = mock(SimpleSlot.class);
+ when(slot.getRoot()).thenReturn(root);
+
+ for (ScheduleMode mode : ScheduleMode.values()) {
+ vertex.getExecutionGraph().setScheduleMode(mode);
+
+ TaskDeploymentDescriptor tdd = vertex.createDeploymentDescriptor(new ExecutionAttemptID(), slot, null, 1);
+
+ Collection<ResultPartitionDeploymentDescriptor> producedPartitions = tdd.getProducedPartitions();
+
+ assertEquals(1, producedPartitions.size());
+ ResultPartitionDeploymentDescriptor desc = producedPartitions.iterator().next();
+ assertEquals(mode.allowLazyDeployment(), desc.sendScheduleOrUpdateConsumersMessage());
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
new file mode 100644
index 0000000..f6fddfa
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.io.network.partition;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.util.TestBufferFactory;
+import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.junit.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class ResultPartitionTest {
+
+ /**
+ * Tests the schedule or update consumers message sending behaviour depending on the relevant flags.
+ */
+ @Test
+ public void testSendScheduleOrUpdateConsumersMessage() throws Exception {
+ {
+ // Pipelined, send message => notify
+ ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+ ResultPartition partition = createPartition(notifier, ResultPartitionType.PIPELINED, true);
+ partition.add(TestBufferFactory.createBuffer(), 0);
+ verify(notifier, times(1)).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+ }
+
+ {
+ // Pipelined, don't send message => don't notify
+ ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+ ResultPartition partition = createPartition(notifier, ResultPartitionType.PIPELINED, false);
+ partition.add(TestBufferFactory.createBuffer(), 0);
+ verify(notifier, never()).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+ }
+
+ {
+ // Blocking, send message => don't notify
+ ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+ ResultPartition partition = createPartition(notifier, ResultPartitionType.BLOCKING, true);
+ partition.add(TestBufferFactory.createBuffer(), 0);
+ verify(notifier, never()).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+ }
+
+ {
+ // Blocking, don't send message => don't notify
+ ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class);
+ ResultPartition partition = createPartition(notifier, ResultPartitionType.BLOCKING, false);
+ partition.add(TestBufferFactory.createBuffer(), 0);
+ verify(notifier, never()).notifyPartitionConsumable(any(JobID.class), any(ResultPartitionID.class), any(TaskActions.class));
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ private static ResultPartition createPartition(
+ ResultPartitionConsumableNotifier notifier,
+ ResultPartitionType type,
+ boolean sendScheduleOrUpdateConsumersMessage) {
+ return new ResultPartition(
+ "TestTask",
+ mock(TaskActions.class),
+ new JobID(),
+ new ResultPartitionID(),
+ type,
+ 1,
+ mock(ResultPartitionManager.class),
+ notifier,
+ mock(IOManager.class),
+ IOManager.IOMode.SYNC,
+ sendScheduleOrUpdateConsumersMessage);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index 2d3797d..4ca1d1f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -124,7 +124,8 @@ public class LocalInputChannelTest {
partitionManager,
partitionConsumableNotifier,
ioManager,
- ASYNC);
+ ASYNC,
+ true);
// Create a buffer pool for this partition
partition.registerBufferPool(
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/ScheduleModeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/ScheduleModeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/ScheduleModeTest.java
new file mode 100644
index 0000000..144ef12
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/ScheduleModeTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.jobgraph;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ScheduleModeTest {
+
+ /**
+ * Test that schedule modes set the lazy deployment flag correctly.
+ */
+ @Test
+ public void testAllowLazyDeployment() throws Exception {
+ assertTrue(ScheduleMode.LAZY_FROM_SOURCES.allowLazyDeployment());
+ assertFalse(ScheduleMode.EAGER.allowLazyDeployment());
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index 15947f9..22f0c60 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -18,7 +18,11 @@
package org.apache.flink.runtime.taskmanager;
-import akka.actor.*;
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.Kill;
+import akka.actor.Props;
+import akka.actor.Status;
import akka.japi.Creator;
import akka.testkit.JavaTestKit;
import org.apache.flink.api.common.ExecutionConfig;
@@ -31,7 +35,11 @@ import org.apache.flink.runtime.akka.FlinkUntypedActor;
import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.concurrent.CompletableFuture;
import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
-import org.apache.flink.runtime.deployment.*;
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionLocation;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.JobInformation;
@@ -622,7 +630,7 @@ public class TaskManagerTest extends TestLogger {
IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
List<ResultPartitionDeploymentDescriptor> irpdd = new ArrayList<ResultPartitionDeploymentDescriptor>();
- irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1));
+ irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1, true));
InputGateDeploymentDescriptor ircdd =
new InputGateDeploymentDescriptor(
@@ -767,7 +775,7 @@ public class TaskManagerTest extends TestLogger {
IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
List<ResultPartitionDeploymentDescriptor> irpdd = new ArrayList<ResultPartitionDeploymentDescriptor>();
- irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1));
+ irpdd.add(new ResultPartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1, true));
InputGateDeploymentDescriptor ircdd =
new InputGateDeploymentDescriptor(
@@ -1419,7 +1427,8 @@ public class TaskManagerTest extends TestLogger {
new IntermediateDataSetID(),
new IntermediateResultPartitionID(),
ResultPartitionType.PIPELINED,
- 1);
+ 1,
+ true);
final TaskDeploymentDescriptor tdd = createTaskDeploymentDescriptor(jid, "TestJob", vid, eid, executionConfig,
"TestTask", 1, 0, 1, 0, new Configuration(), new Configuration(),
http://git-wip-us.apache.org/repos/asf/flink/blob/2742d5c1/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 1eebe12..5d26050 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -553,6 +553,9 @@ public class TaskTest extends TestLogger {
}
expected.put(ExecutionState.RUNNING, ExecutionState.RUNNING);
+ expected.put(ExecutionState.SCHEDULED, ExecutionState.RUNNING);
+ expected.put(ExecutionState.DEPLOYING, ExecutionState.RUNNING);
+ expected.put(ExecutionState.FINISHED, ExecutionState.RUNNING);
expected.put(ExecutionState.CANCELED, ExecutionState.CANCELING);
expected.put(ExecutionState.CANCELING, ExecutionState.CANCELING);
@@ -568,7 +571,7 @@ public class TaskTest extends TestLogger {
assertEquals(expected.get(state), newTaskState);
}
- verify(inputGate, times(1)).retriggerPartitionRequest(eq(partitionId.getPartitionId()));
+ verify(inputGate, times(4)).retriggerPartitionRequest(eq(partitionId.getPartitionId()));
}
/**