From 198028f4ff1cc01e0e39bd389937a23b9ea45c7f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 29 Aug 2016 16:58:31 +0200 Subject: [PATCH 1/4] [FLINK-4490] [distributed coordination] (part 1) Change InstanceConnectionInfo to TaskManagerLocation This adds the ResourceId to the TaskManagerLocation --- .../handlers/JobExceptionsHandler.java | 4 +- .../handlers/JobVertexDetailsHandler.java | 4 +- .../JobVertexTaskManagersHandler.java | 4 +- ...SubtaskExecutionAttemptDetailsHandler.java | 4 +- .../SubtasksAllAccumulatorsHandler.java | 4 +- .../handlers/SubtasksTimesHandler.java | 4 +- ...rtialInputChannelDeploymentDescriptor.java | 10 +- .../runtime/executiongraph/Execution.java | 9 +- .../executiongraph/ExecutionVertex.java | 4 +- .../flink/runtime/instance/Instance.java | 7 +- .../runtime/instance/InstanceManager.java | 3 +- .../runtime/io/network/ConnectionID.java | 4 +- .../io/network/NetworkEnvironment.java | 6 +- .../TaskManagerLocation.java} | 190 +++++++++--------- .../messages/RegistrationMessages.scala | 12 +- .../runtime/taskmanager/TaskManager.scala | 29 +-- .../testingUtils/TestingTaskManager.scala | 33 ++- .../ExecutionGraphMetricsTest.java | 8 +- .../ExecutionGraphTestUtils.java | 7 +- .../TerminalStateDeadlockTest.java | 7 +- .../VertexLocationConstraintTest.java | 6 +- .../runtime/instance/InstanceManagerTest.java | 64 +++--- .../flink/runtime/instance/InstanceTest.java | 16 +- .../runtime/instance/SimpleSlotTest.java | 6 +- .../io/network/NetworkEnvironmentTest.java | 7 +- .../scheduler/SchedulerTestUtils.java | 25 +-- .../testutils/DummyEnvironment.java.orig | 185 ----------------- .../ResourceManagerITCase.java | 5 +- ...kManagerComponentsStartupShutdownTest.java | 6 +- .../TaskManagerLocationTest.java} | 49 ++--- .../JobManagerRegistrationTest.scala | 14 +- .../flink/yarn/TestingYarnTaskManager.scala | 20 +- .../apache/flink/yarn/YarnTaskManager.scala | 20 +- 33 files changed, 291 insertions(+), 485 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/{instance/InstanceConnectionInfo.java => taskmanager/TaskManagerLocation.java} (61%) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java.orig rename flink-runtime/src/test/java/org/apache/flink/runtime/{instance/InstanceConnectionInfoTest.java => taskmanager/TaskManagerLocationTest.java} (75%) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java index 7b6a36191f3fa..ce154e372cc3d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.util.ExceptionUtils; @@ -66,7 +66,7 @@ public String handleRequest(ExecutionGraph graph, Map params) th break; } - InstanceConnectionInfo location = task.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = task.getCurrentAssignedResourceLocation(); String locationString = location != null ? location.getFQDNHostname() + ':' + location.dataPort() : "(unassigned)"; diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java index d4e885e25aa81..813ecb82a260d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -61,7 +61,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { final ExecutionState status = vertex.getExecutionState(); - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort(); long startTime = vertex.getStateTimestamp(ExecutionState.DEPLOYING); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java index befc0bfa6509a..cbdb87f307444 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -51,7 +51,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa Map> taskManagerVertices = new HashMap<>(); for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String taskManager = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort(); List vertices = taskManagerVertices.get(taskManager); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java index 3d80b235c86f0..a1e6d0ed29679 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -45,7 +45,7 @@ public String handleRequest(Execution execAttempt, Map params) t final ExecutionState status = execAttempt.getState(); final long now = System.currentTimeMillis(); - InstanceConnectionInfo location = execAttempt.getAssignedResourceLocation(); + TaskManagerLocation location = execAttempt.getAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); long startTime = execAttempt.getStateTimestamp(ExecutionState.DEPLOYING); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java index 6d9ce3a49e15b..780bd4b696a99 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -52,7 +52,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa int num = 0; for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); gen.writeStartObject(); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java index 03d40dc12e3dd..9e6276d1312e2 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -70,7 +70,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa gen.writeStartObject(); gen.writeNumberField("subtask", num++); - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); gen.writeStringField("host", locationString); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java index a3cfcd934d718..e1391a46d7ea2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.IntermediateResult; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -44,7 +44,7 @@ public class PartialInputChannelDeploymentDescriptor { private final ResultPartitionID partitionID; /** The partition connection info. */ - private final InstanceConnectionInfo partitionConnectionInfo; + private final TaskManagerLocation partitionConnectionInfo; /** The partition connection index. */ private final int partitionConnectionIndex; @@ -52,7 +52,7 @@ public class PartialInputChannelDeploymentDescriptor { public PartialInputChannelDeploymentDescriptor( IntermediateDataSetID resultId, ResultPartitionID partitionID, - InstanceConnectionInfo partitionConnectionInfo, + TaskManagerLocation partitionConnectionInfo, int partitionConnectionIndex) { this.resultId = checkNotNull(resultId); @@ -71,7 +71,7 @@ public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor( checkNotNull(consumerExecution, "Consumer execution null"); - InstanceConnectionInfo consumerConnectionInfo = consumerExecution.getAssignedResourceLocation(); + TaskManagerLocation consumerConnectionInfo = consumerExecution.getAssignedResourceLocation(); checkNotNull(consumerConnectionInfo, "Consumer connection info null"); @@ -107,7 +107,7 @@ public static PartialInputChannelDeploymentDescriptor fromEdge( final IntermediateResult result = partition.getIntermediateResult(); final IntermediateDataSetID resultId = result.getId(); - final InstanceConnectionInfo partitionConnectionInfo = producer.getAssignedResourceLocation(); + final TaskManagerLocation partitionConnectionInfo = producer.getAssignedResourceLocation(); final int partitionConnectionIndex = result.getConnectionIndex(); return new PartialInputChannelDeploymentDescriptor( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 5bab780da9928..16f5d34b85bce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -31,7 +31,7 @@ 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.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.ConnectionID; @@ -47,7 +47,6 @@ import org.apache.flink.runtime.messages.Messages; import org.apache.flink.runtime.messages.TaskMessages.TaskOperationResult; import org.apache.flink.runtime.state.StateHandle; -import org.apache.flink.runtime.util.SerializableObject; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.ExceptionUtils; @@ -132,7 +131,7 @@ public class Execution { private volatile Throwable failureCause; // once assigned, never changes - private volatile InstanceConnectionInfo assignedResourceLocation; // for the archived execution + private volatile TaskManagerLocation assignedResourceLocation; // for the archived execution /** The state with which the execution attempt should start */ private SerializedValue> operatorState; @@ -144,7 +143,7 @@ public class Execution { /* Lock for updating the accumulators atomically. Prevents final accumulators to be overwritten * by partial accumulators on a late heartbeat*/ - private final SerializableObject accumulatorLock = new SerializableObject(); + private final Object accumulatorLock = new Object(); /* Continuously updated map of user-defined accumulators */ private volatile Map> userAccumulators; @@ -199,7 +198,7 @@ public SimpleSlot getAssignedResource() { return assignedResource; } - public InstanceConnectionInfo getAssignedResourceLocation() { + public TaskManagerLocation getAssignedResourceLocation() { return assignedResourceLocation; } 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 2495316dc9c32..f74a3491e93ce 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 @@ -28,7 +28,7 @@ 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.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -222,7 +222,7 @@ public SimpleSlot getCurrentAssignedResource() { return currentExecution.getAssignedResource(); } - public InstanceConnectionInfo getCurrentAssignedResourceLocation() { + public TaskManagerLocation getCurrentAssignedResourceLocation() { return currentExecution.getAssignedResourceLocation(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index 119f060315835..598b32bbc89ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +47,7 @@ public class Instance { private final ActorGateway actorGateway; /** The instance connection information for the data transfer. */ - private final InstanceConnectionInfo connectionInfo; + private final TaskManagerLocation connectionInfo; /** A description of the resources of the task manager */ private final HardwareDescription resources; @@ -92,7 +93,7 @@ public class Instance { */ public Instance( ActorGateway actorGateway, - InstanceConnectionInfo connectionInfo, + TaskManagerLocation connectionInfo, ResourceID resourceId, InstanceID id, HardwareDescription resources, @@ -350,7 +351,7 @@ public ActorGateway getActorGateway() { return actorGateway; } - public InstanceConnectionInfo getInstanceConnectionInfo() { + public TaskManagerLocation getInstanceConnectionInfo() { return connectionInfo; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index 0d0d4c7ff6923..e7a45376d4631 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -29,6 +29,7 @@ import akka.actor.ActorRef; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -147,7 +148,7 @@ public boolean reportHeartBeat(InstanceID instanceId, byte[] lastMetricsReport) public InstanceID registerTaskManager( ActorRef taskManager, ResourceID resourceID, - InstanceConnectionInfo connectionInfo, + TaskManagerLocation connectionInfo, HardwareDescription resources, int numberOfSlots, UUID leaderSessionID){ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java index c15e72e73457a..b5eab9086366f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network; import org.apache.flink.runtime.executiongraph.IntermediateResult; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.io.Serializable; import java.net.InetSocketAddress; @@ -41,7 +41,7 @@ public class ConnectionID implements Serializable { private final int connectionIndex; - public ConnectionID(InstanceConnectionInfo connectionInfo, int connectionIndex) { + public ConnectionID(TaskManagerLocation connectionInfo, int connectionIndex) { this(new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()), connectionIndex); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index 844bc2dadea2c..aa428f202eb8b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -109,7 +109,7 @@ public class NetworkEnvironment { */ private final ExecutionContext executionContext; - private final InstanceConnectionInfo connectionInfo; + private final TaskManagerLocation connectionInfo; /** * Initializes all network I/O components. @@ -118,7 +118,7 @@ public NetworkEnvironment( ExecutionContext executionContext, FiniteDuration jobManagerTimeout, NetworkEnvironmentConfiguration config, - InstanceConnectionInfo connectionInfo) throws IOException { + TaskManagerLocation connectionInfo) throws IOException { this.executionContext = executionContext; this.configuration = checkNotNull(config); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java similarity index 61% rename from flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index eb87292afa8ce..e39d7d285e486 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -16,53 +16,53 @@ * limitations under the License. */ -package org.apache.flink.runtime.instance; +package org.apache.flink.runtime.taskmanager; -import java.io.IOException; import java.net.InetAddress; -import java.net.UnknownHostException; -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.StringUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * This class encapsulates the connection information of a TaskManager. * It describes the host where the TaskManager operates and its server port * for data exchange. This class also contains utilities to work with the * TaskManager's host name, which is used to localize work assignments. */ -public class InstanceConnectionInfo implements IOReadableWritable, Comparable, java.io.Serializable { +public class TaskManagerLocation implements Comparable, java.io.Serializable { private static final long serialVersionUID = -8254407801276350716L; - - private static final Logger LOG = LoggerFactory.getLogger(InstanceConnectionInfo.class); - - /** - * The network address the instance's task manager binds its sockets to. - */ - private InetAddress inetAddress; + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerLocation.class); - /** - * The port the instance's task manager expects to receive transfer envelopes on. - */ - private int dataPort; + // ------------------------------------------------------------------------ - /** - * The fully qualified host name of the instance. - */ - private String fqdnHostName; + /** The ID of the resource in which the TaskManager is started. This can be for example + * the YARN container ID, Mesos container ID, or any other unique identifier. */ + private final ResourceID resourceID; + + /** The network address that the TaskManager binds its sockets to */ + private final InetAddress inetAddress; + + /** The fully qualified host name of the TaskManager */ + private final String fqdnHostName; + + /** The pure hostname, derived from the fully qualified host name. */ + private final String hostName; - /** - * The hostname, derived from the fully qualified host name. - */ - private String hostName; + /** The port that the TaskManager receive data transport connection requests at */ + private final int dataPort; + /** The toString representation, eagerly constructed and cached to avoid repeated string building */ + private final String stringRepresentation; /** * Constructs a new instance connection info object. The constructor will attempt to retrieve the instance's @@ -73,27 +73,25 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable 0, "dataPort must be > 0"); + this.resourceID = checkNotNull(resourceID); + this.inetAddress = checkNotNull(inetAddress); this.dataPort = dataPort; - this.inetAddress = inetAddress; - + // get FQDN hostname on this TaskManager. + String fqdnHostName; try { - this.fqdnHostName = this.inetAddress.getCanonicalHostName(); + fqdnHostName = this.inetAddress.getCanonicalHostName(); } catch (Throwable t) { LOG.warn("Unable to determine the canonical hostname. Input split assignment (such as " + "for HDFS files) may be non-local when the canonical hostname is missing."); LOG.debug("getCanonicalHostName() Exception:", t); - this.fqdnHostName = this.inetAddress.getHostAddress(); + fqdnHostName = this.inetAddress.getHostAddress(); } + this.fqdnHostName = fqdnHostName; if (this.fqdnHostName.equals(this.inetAddress.getHostAddress())) { // this happens when the name lookup fails, either due to an exception, @@ -107,13 +105,30 @@ public InstanceConnectionInfo(InetAddress inetAddress, int dataPort) { else { this.hostName = NetUtils.getHostnameFromFQDN(this.fqdnHostName); } + + this.stringRepresentation = String.format( + "TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); } + // ------------------------------------------------------------------------ + // Getters + // ------------------------------------------------------------------------ + /** - * Constructs an empty object. + * Gets the ID of the resource in which the TaskManager is started. The format of this depends + * on how the TaskManager is started: + *
    + *
  • If the TaskManager is started via YARN, this is the YARN container ID.
  • + *
  • If the TaskManager is started via Mesos, this is the Mesos container ID.
  • + *
  • If the TaskManager is started in standalone mode, or via a MiniCluster, this is a random ID.
  • + *
  • Other deployment modes can set the resource ID in other ways.
  • + *
+ * + * @return The ID of the resource in which the TaskManager is started */ - public InstanceConnectionInfo() {} - + public ResourceID getResourceID() { + return resourceID; + } /** * Returns the port instance's task manager expects to receive transfer envelopes on. @@ -121,7 +136,7 @@ public InstanceConnectionInfo() {} * @return the port instance's task manager expects to receive transfer envelopes on */ public int dataPort() { - return this.dataPort; + return dataPort; } /** @@ -130,7 +145,16 @@ public int dataPort() { * @return the network address the instance's task manager binds its sockets to */ public InetAddress address() { - return this.inetAddress; + return inetAddress; + } + + /** + * Gets the IP address where the TaskManager operates. + * + * @return The IP address. + */ + public String addressString() { + return inetAddress.toString(); } /** @@ -140,7 +164,7 @@ public InetAddress address() { * @return The fully-qualified domain name of the TaskManager. */ public String getFQDNHostname() { - return this.fqdnHostName; + return fqdnHostName; } /** @@ -160,82 +184,50 @@ public String getHostname() { return hostName; } - /** - * Gets the IP address where the TaskManager operates. - * - * @return The IP address. - */ - public String getInetAdress() { - return this.inetAddress.toString(); - } - - // -------------------------------------------------------------------------------------------- - // Serialization - // -------------------------------------------------------------------------------------------- - - @Override - public void read(DataInputView in) throws IOException { - - final int addr_length = in.readInt(); - byte[] address = new byte[addr_length]; - in.readFully(address); - - this.dataPort = in.readInt(); - - this.fqdnHostName = StringUtils.readNullableString(in); - this.hostName = StringUtils.readNullableString(in); - - try { - this.inetAddress = InetAddress.getByAddress(address); - } catch (UnknownHostException e) { - throw new IOException("This lookup should never fail.", e); - } - } - - - @Override - public void write(final DataOutputView out) throws IOException { - out.writeInt(this.inetAddress.getAddress().length); - out.write(this.inetAddress.getAddress()); - - out.writeInt(this.dataPort); - - StringUtils.writeNullableString(fqdnHostName, out); - StringUtils.writeNullableString(hostName, out); - } - // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @Override public String toString() { - return getFQDNHostname() + " (dataPort=" + dataPort + ")"; + return stringRepresentation; } @Override public boolean equals(Object obj) { - if (obj instanceof InstanceConnectionInfo) { - InstanceConnectionInfo other = (InstanceConnectionInfo) obj; - return this.dataPort == other.dataPort && - this.inetAddress.equals(other.inetAddress); - } else { + if (obj == this) { + return true; + } + else if (obj != null && obj.getClass() == TaskManagerLocation.class) { + TaskManagerLocation that = (TaskManagerLocation) obj; + return this.resourceID.equals(that.resourceID) && + this.inetAddress.equals(that.inetAddress) && + this.dataPort == that.dataPort; + } + else { return false; } } @Override public int hashCode() { - return this.inetAddress.hashCode() + - 17*dataPort; + return resourceID.hashCode() + + 17 * inetAddress.hashCode() + + 129 * dataPort; } @Override - public int compareTo(InstanceConnectionInfo o) { + public int compareTo(@Nonnull TaskManagerLocation o) { // decide based on address first + int resourceIdCmp = this.resourceID.getResourceIdString().compareTo(o.resourceID.getResourceIdString()); + if (resourceIdCmp != 0) { + return resourceIdCmp; + } + + // decide based on ip address next byte[] thisAddress = this.inetAddress.getAddress(); byte[] otherAddress = o.inetAddress.getAddress(); - + if (thisAddress.length < otherAddress.length) { return -1; } else if (thisAddress.length > otherAddress.length) { @@ -251,7 +243,7 @@ public int compareTo(InstanceConnectionInfo o) { } } } - + // addresses are identical, decide based on ports. if (this.dataPort < o.dataPort) { return -1; diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala index d362164eb4ce6..5648bc69f9e5f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala @@ -20,9 +20,9 @@ package org.apache.flink.runtime.messages import java.util.UUID -import akka.actor.ActorRef import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.{HardwareDescription, InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance.{HardwareDescription, InstanceID} +import org.apache.flink.runtime.taskmanager.TaskManagerLocation import scala.concurrent.duration.{Deadline, FiniteDuration} @@ -63,10 +63,10 @@ object RegistrationMessages { * @param numberOfSlots The number of processing slots offered by the TaskManager. */ case class RegisterTaskManager( - resourceId: ResourceID, - connectionInfo: InstanceConnectionInfo, - resources: HardwareDescription, - numberOfSlots: Int) + resourceId: ResourceID, + connectionInfo: TaskManagerLocation, + resources: HardwareDescription, + numberOfSlots: Int) extends RegistrationMessage /** diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 72ec2ac4474f7..1bea7530326ec 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -50,7 +50,7 @@ import org.apache.flink.runtime.execution.ExecutionState import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} import org.apache.flink.runtime.executiongraph.ExecutionAttemptID import org.apache.flink.runtime.filecache.FileCache -import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance._ import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} import org.apache.flink.runtime.io.network.NetworkEnvironment @@ -122,14 +122,14 @@ import scala.util.{Failure, Success} * requires a clean JVM. */ class TaskManager( - protected val config: TaskManagerConfiguration, - protected val resourceID: ResourceID, - protected val connectionInfo: InstanceConnectionInfo, - protected val memoryManager: MemoryManager, - protected val ioManager: IOManager, - protected val network: NetworkEnvironment, - protected val numberOfSlots: Int, - protected val leaderRetrievalService: LeaderRetrievalService) + protected val config: TaskManagerConfiguration, + protected val resourceID: ResourceID, + protected val connectionInfo: TaskManagerLocation, + protected val memoryManager: MemoryManager, + protected val ioManager: IOManager, + protected val network: NetworkEnvironment, + protected val numberOfSlots: Int, + protected val leaderRetrievalService: LeaderRetrievalService) extends FlinkActor with LeaderSessionMessageFilter // Mixin order is important: We want to filter after logging with LogMessages // Mixin order is important: first we want to support message logging @@ -1796,11 +1796,12 @@ object TaskManager { val (taskManagerConfig : TaskManagerConfiguration, netConfig: NetworkEnvironmentConfiguration, - connectionInfo: InstanceConnectionInfo, + connectionInfo: TaskManagerLocation, memType: MemoryType ) = parseTaskManagerConfiguration( configuration, taskManagerHostname, + resourceID, localTaskManagerCommunication) // pre-start checks @@ -1979,19 +1980,21 @@ object TaskManager { * * @param configuration The configuration. * @param taskManagerHostname The host name under which the TaskManager communicates. + * @param resourceID The TaskManager's Resource ID * @param localTaskManagerCommunication True, to skip initializing the network stack. * Use only in cases where only one task manager runs. * @return A tuple (TaskManagerConfiguration, network configuration, - * InstanceConnectionInfo, JobManager actor Akka URL). + * TaskManagerLocation, JobManager actor Akka URL). */ @throws(classOf[IllegalArgumentException]) def parseTaskManagerConfiguration( configuration: Configuration, taskManagerHostname: String, + resourceID: ResourceID, localTaskManagerCommunication: Boolean) : (TaskManagerConfiguration, NetworkEnvironmentConfiguration, - InstanceConnectionInfo, + TaskManagerLocation, MemoryType) = { // ------- read values from the config and check them --------- @@ -2009,7 +2012,7 @@ object TaskManager { "Leave config parameter empty or use 0 to let the system choose a port automatically.") val taskManagerAddress = InetAddress.getByName(taskManagerHostname) - val connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport) + val connectionInfo = new TaskManagerLocation(resourceID, taskManagerAddress, dataport) // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala index 2597753e15f90..9b5a147360389 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala @@ -19,26 +19,25 @@ package org.apache.flink.runtime.testingUtils import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration} import scala.language.postfixOps /** Subclass of the [[TaskManager]] to support testing messages */ class TestingTaskManager( - config: TaskManagerConfiguration, - resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) + config: TaskManagerConfiguration, + resourceID: ResourceID, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) extends TaskManager( config, resourceID, @@ -51,13 +50,13 @@ class TestingTaskManager( with TestingTaskManagerLike { def this( - config: TaskManagerConfiguration, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) { + config: TaskManagerConfiguration, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) { this( config, ResourceID.generate(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index 2b8b867d5ee0f..cf7cf5800ec7c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -110,7 +110,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti Instance instance = mock(Instance.class); - InstanceConnectionInfo instanceConnectionInfo = mock(InstanceConnectionInfo.class); + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); Slot rootSlot = mock(Slot.class); @@ -123,9 +123,9 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); - when(instance.getInstanceConnectionInfo()).thenReturn(instanceConnectionInfo); + when(instance.getInstanceConnectionInfo()).thenReturn(taskManagerLocation); when(instance.getActorGateway()).thenReturn(actorGateway); - when(instanceConnectionInfo.getHostname()).thenReturn("localhost"); + when(taskManagerLocation.getHostname()).thenReturn("localhost"); when(rootSlot.getSlotNumber()).thenReturn(0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 903d5f943be2c..cddb6cbf61502 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.instance.BaseTestingActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; @@ -108,11 +108,12 @@ public static Instance getInstance(final ActorGateway gateway) throws Exception } public static Instance getInstance(final ActorGateway gateway, final int numberOfSlots) throws Exception { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); - return new Instance(gateway, connection, ResourceID.generate(), new InstanceID(), hardwareDescription, numberOfSlots); + return new Instance(gateway, connection, resourceID, new InstanceID(), hardwareDescription, numberOfSlots); } @SuppressWarnings("serial") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index 2a690d99359a4..a71faf62eedc8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -79,12 +79,13 @@ public TerminalStateDeadlockTest() { this.execGraphSchedulerField.setAccessible(true); // the dummy resource + ResourceID resourceId = ResourceID.generate(); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ci = new InstanceConnectionInfo(address, 12345); + TaskManagerLocation ci = new TaskManagerLocation(resourceId, address, 12345); HardwareDescription resources = new HardwareDescription(4, 4000000, 3000000, 2000000); Instance instance = new Instance(DummyActorGateway.INSTANCE, ci, - ResourceID.generate(), new InstanceID(), resources, 4); + resourceId, new InstanceID(), resources, 4); this.resource = instance.allocateSimpleSlot(new JobID()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 4ee06b364c4ba..91472ae2fdbb5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -443,10 +443,10 @@ public void testArchivingClearsFields() { public static Instance getInstance(byte[] ipAddress, int dataPort, String hostname) throws Exception { HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); - InstanceConnectionInfo connection = mock(InstanceConnectionInfo.class); + TaskManagerLocation connection = mock(TaskManagerLocation.class); when(connection.address()).thenReturn(InetAddress.getByAddress(ipAddress)); when(connection.dataPort()).thenReturn(dataPort); - when(connection.getInetAdress()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); + when(connection.addressString()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); when(connection.getHostname()).thenReturn(hostname); when(connection.getFQDNHostname()).thenReturn(hostname); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java index ff5e2ab4ede96..f1ed9601e8ec3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java @@ -18,24 +18,11 @@ package org.apache.flink.runtime.instance; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -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 java.net.InetAddress; -import java.util.Collection; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import java.util.UUID; - import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; + import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -44,6 +31,18 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.net.InetAddress; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Tests for {@link org.apache.flink.runtime.instance.InstanceManager}. */ @@ -76,13 +75,13 @@ public void testInstanceRegistering() { InetAddress address = InetAddress.getByName("127.0.0.1"); // register three instances - InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, dataPort); - InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, dataPort + 15); - InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, dataPort + 30); - ResourceID resID1 = ResourceID.generate(); ResourceID resID2 = ResourceID.generate(); ResourceID resID3 = ResourceID.generate(); + + TaskManagerLocation ici1 = new TaskManagerLocation(resID1, address, dataPort); + TaskManagerLocation ici2 = new TaskManagerLocation(resID2, address, dataPort + 15); + TaskManagerLocation ici3 = new TaskManagerLocation(resID3, address, dataPort + 30); final JavaTestKit probe1 = new JavaTestKit(system); final JavaTestKit probe2 = new JavaTestKit(system); @@ -99,16 +98,16 @@ public void testInstanceRegistering() { assertEquals(8, cm.getTotalNumberOfSlots()); Collection instances = cm.getAllRegisteredInstances(); - Set instanceConnectionInfos = new - HashSet(); + Set taskManagerLocations = new + HashSet(); for(Instance instance: instances){ - instanceConnectionInfos.add(instance.getInstanceConnectionInfo()); + taskManagerLocations.add(instance.getInstanceConnectionInfo()); } - assertTrue(instanceConnectionInfos.contains(ici1)); - assertTrue(instanceConnectionInfos.contains(ici2)); - assertTrue(instanceConnectionInfos.contains(ici3)); + assertTrue(taskManagerLocations.contains(ici1)); + assertTrue(taskManagerLocations.contains(ici2)); + assertTrue(taskManagerLocations.contains(ici3)); cm.shutdown(); } @@ -131,7 +130,7 @@ public void testRegisteringAlreadyRegistered() { HardwareDescription resources = HardwareDescription.extractFromSystem(4096); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ici = new InstanceConnectionInfo(address, dataPort); + TaskManagerLocation ici = new TaskManagerLocation(resID1, address, dataPort); JavaTestKit probe = new JavaTestKit(system); cm.registerTaskManager(probe.getRef(), resID1, @@ -141,13 +140,12 @@ public void testRegisteringAlreadyRegistered() { assertEquals(1, cm.getTotalNumberOfSlots()); try { - cm.registerTaskManager(probe.getRef(), resID2, - ici, resources, 1, leaderSessionID); + cm.registerTaskManager(probe.getRef(), resID2, ici, resources, 1, leaderSessionID); } catch (Exception e) { // good } - // check for correct number of registerede instances + // check for correct number of registered instances assertEquals(1, cm.getNumberOfRegisteredTaskManagers()); assertEquals(1, cm.getTotalNumberOfSlots()); @@ -176,9 +174,9 @@ public void testReportHeartbeat() { InetAddress address = InetAddress.getByName("127.0.0.1"); // register three instances - InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, dataPort); - InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, dataPort + 1); - InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, dataPort + 2); + TaskManagerLocation ici1 = new TaskManagerLocation(resID1, address, dataPort); + TaskManagerLocation ici2 = new TaskManagerLocation(resID2, address, dataPort + 1); + TaskManagerLocation ici3 = new TaskManagerLocation(resID3, address, dataPort + 2); JavaTestKit probe1 = new JavaTestKit(system); JavaTestKit probe2 = new JavaTestKit(system); @@ -240,7 +238,7 @@ public void testShutdown() { ResourceID resID = ResourceID.generate(); HardwareDescription resources = HardwareDescription.extractFromSystem(4096); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ici = new InstanceConnectionInfo(address, 20000); + TaskManagerLocation ici = new TaskManagerLocation(resID, address, 20000); JavaTestKit probe = new JavaTestKit(system); cm.registerTaskManager(probe.getRef(), resID, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java index faa679b6337f8..82d3723b92d98 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.junit.Test; /** @@ -35,12 +36,13 @@ public class InstanceTest { @Test public void testAllocatingAndCancellingSlots() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 4); + resourceID, new InstanceID(), hardwareDescription, 4); assertEquals(4, instance.getTotalNumberOfSlots()); assertEquals(4, instance.getNumberOfAvailableSlots()); @@ -97,12 +99,13 @@ public void testAllocatingAndCancellingSlots() { @Test public void testInstanceDies() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 3); + resourceID, new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); @@ -128,12 +131,13 @@ public void testInstanceDies() { @Test public void testCancelAllSlots() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 3); + resourceID, new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java index 459a3ed613096..82c2a740cd620 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.junit.Test; import org.mockito.Matchers; @@ -143,12 +144,13 @@ public void testReleaseCancelsVertex() { } public static SimpleSlot getSlot() throws Exception { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 1); + resourceID, new InstanceID(), hardwareDescription, 1); return instance.allocateSimpleSlot(new JobID()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java index 4597e3bf2e897..2be07d043b420 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java @@ -21,9 +21,10 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.DummyActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -87,7 +88,7 @@ public void testAssociateDisassociate() { TestingUtils.defaultExecutionContext(), new FiniteDuration(30, TimeUnit.SECONDS), config, - new InstanceConnectionInfo(InetAddress.getLocalHost(), port)); + new TaskManagerLocation(ResourceID.generate(), InetAddress.getLocalHost(), port)); assertFalse(env.isShutdown()); assertFalse(env.isAssociated()); @@ -184,7 +185,7 @@ public void testEagerlyDeployConsumers() throws Exception { TestingUtils.defaultExecutionContext(), new FiniteDuration(30, TimeUnit.SECONDS), config, - new InstanceConnectionInfo(InetAddress.getLocalHost(), 12232)); + new TaskManagerLocation(ResourceID.generate(), InetAddress.getLocalHost(), 12232)); // Associate the environment with the mock actors env.associateWithTaskManagerAndJobManager( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index ed6d1ee5720e3..1736c390bdb99 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -24,7 +24,6 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.concurrent.atomic.AtomicInteger; @@ -35,7 +34,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -52,7 +51,8 @@ public static Instance getRandomInstance(int numSlots) { throw new IllegalArgumentException(); } - InetAddress address; + final ResourceID resourceID = ResourceID.generate(); + final InetAddress address; try { address = InetAddress.getByName("127.0.0.1"); } @@ -62,12 +62,12 @@ public static Instance getRandomInstance(int numSlots) { int dataPort = port.getAndIncrement(); - InstanceConnectionInfo ci = new InstanceConnectionInfo(address, dataPort); + TaskManagerLocation ci = new TaskManagerLocation(resourceID, address, dataPort); final long GB = 1024L*1024*1024; HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB); - return new Instance(DummyActorGateway.INSTANCE, ci, ResourceID.generate(), + return new Instance(DummyActorGateway.INSTANCE, ci, resourceID, new InstanceID(), resources, numSlots); } @@ -141,19 +141,4 @@ public static boolean areAllDistinct(Object ... obj) { return set.size() == obj.length; } - - public static boolean areSameSets(Collection set1, Collection set2) { - if (set1 == null || set2 == null) { - throw new IllegalArgumentException(); - } - - HashSet set = new HashSet(set1); - for (Object o : set2) { - if (!set.remove(o)) { - return false; - } - } - - return set.isEmpty(); - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java.orig b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java.orig deleted file mode 100644 index 393ee4c920223..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java.orig +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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.operators.testutils; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.metrics.groups.TaskMetricGroup; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.broadcast.BroadcastVariableManager; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.StateHandle; -import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; - -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.Future; - -public class DummyEnvironment implements Environment { - - private final JobID jobId = new JobID(); - private final JobVertexID jobVertexId = new JobVertexID(); - private final ExecutionAttemptID executionId = new ExecutionAttemptID(); - private final ExecutionConfig executionConfig = new ExecutionConfig(); -<<<<<<< 9a73dbc71b83080b7deccc62b8b6ffa9f102e847 - private final TaskInfo taskInfo; -======= - private final KvStateRegistry kvStateRegistry = new KvStateRegistry(); - private final TaskKvStateRegistry taskKvStateRegistry; ->>>>>>> [FLINK-3779] [runtime] Add KvStateRegistry for queryable KvState - - public DummyEnvironment(String taskName, int numSubTasks, int subTaskIndex) { - this.taskInfo = new TaskInfo(taskName, subTaskIndex, numSubTasks, 0); - - this.taskKvStateRegistry = kvStateRegistry.createTaskRegistry(jobId, jobVertexId); - } - - public KvStateRegistry getKvStateRegistry() { - return kvStateRegistry; - } - - @Override - public ExecutionConfig getExecutionConfig() { - return executionConfig; - } - - @Override - public JobID getJobID() { - return jobId; - } - - @Override - public JobVertexID getJobVertexId() { - return jobVertexId; - } - - @Override - public ExecutionAttemptID getExecutionId() { - return executionId; - } - - @Override - public Configuration getTaskConfiguration() { - return new Configuration(); - } - - @Override - public TaskManagerRuntimeInfo getTaskManagerInfo() { - return null; - } - - @Override - public TaskMetricGroup getMetricGroup() { - return new UnregisteredTaskMetricsGroup(); - } - - @Override - public Configuration getJobConfiguration() { - return new Configuration(); - } - - @Override - public TaskInfo getTaskInfo() { - return taskInfo; - } - - @Override - public InputSplitProvider getInputSplitProvider() { - return null; - } - - @Override - public IOManager getIOManager() { - return null; - } - - @Override - public MemoryManager getMemoryManager() { - return null; - } - - @Override - public ClassLoader getUserClassLoader() { - return getClass().getClassLoader(); - } - - @Override - public Map> getDistributedCacheEntries() { - return Collections.emptyMap(); - } - - @Override - public BroadcastVariableManager getBroadcastVariableManager() { - return null; - } - - @Override - public AccumulatorRegistry getAccumulatorRegistry() { - return null; - } - - @Override -<<<<<<< 9a73dbc71b83080b7deccc62b8b6ffa9f102e847 - public void acknowledgeCheckpoint(long checkpointId) {} -======= - public TaskKvStateRegistry getTaskKvStateRegistry() { - return taskKvStateRegistry; - } - - @Override - public void acknowledgeCheckpoint(long checkpointId) { - } ->>>>>>> [FLINK-3779] [runtime] Add KvStateRegistry for queryable KvState - - @Override - public void acknowledgeCheckpoint(long checkpointId, StateHandle state) {} - - @Override - public ResultPartitionWriter getWriter(int index) { - return null; - } - - @Override - public ResultPartitionWriter[] getAllWriters() { - return null; - } - - @Override - public InputGate getInputGate(int index) { - return null; - } - - @Override - public InputGate[] getAllInputGates() { - return null; - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java index ca09634a79124..3307568bf99ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java @@ -24,14 +24,13 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.messages.Messages; import org.apache.flink.runtime.messages.RegistrationMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.TestingResourceManager; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; @@ -83,7 +82,7 @@ protected void run() { jobManager.tell( new RegistrationMessages.RegisterTaskManager( resourceID, - Mockito.mock(InstanceConnectionInfo.class), + Mockito.mock(TaskManagerLocation.class), null, 1), me); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 147a3e0accef5..458c834becc1a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -32,7 +32,6 @@ import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.NetworkEnvironment; @@ -101,7 +100,8 @@ public void testComponentsStartupShutdown() { 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0, Option.empty(), new Tuple2(0, 0)); - final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000); + final ResourceID resourceID = ResourceID.generate(); + final TaskManagerLocation connectionInfo = new TaskManagerLocation(resourceID, InetAddress.getLocalHost(), 10000); final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false); final IOManager ioManager = new IOManagerAsync(TMP_DIR); @@ -118,7 +118,7 @@ public void testComponentsStartupShutdown() { final Props tmProps = Props.create( TaskManager.class, tmConfig, - ResourceID.generate(), + resourceID, connectionInfo, memManager, ioManager, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java similarity index 75% rename from flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java index 3a9488d2a0a1b..9452b20fee967 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.instance; +package org.apache.flink.runtime.taskmanager; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertEquals; @@ -25,6 +25,7 @@ import java.net.InetAddress; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; @@ -34,14 +35,18 @@ import static org.mockito.Mockito.when; /** - * Tests for the InstanceConnectionInfo, which identifies the location and connection + * Tests for the TaskManagerLocation, which identifies the location and connection * information of a TaskManager. */ -public class InstanceConnectionInfoTest { +public class TaskManagerLocationTest { @Test public void testEqualsHashAndCompareTo() { try { + ResourceID resourceID1 = new ResourceID("a"); + ResourceID resourceID2 = new ResourceID("b"); + ResourceID resourceID3 = new ResourceID("c"); + // we mock the addresses to save the times of the reverse name lookups InetAddress address1 = mock(InetAddress.class); when(address1.getCanonicalHostName()).thenReturn("localhost"); @@ -62,10 +67,10 @@ public void testEqualsHashAndCompareTo() { when(address3.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, 0, 1} ); // one == four != two != three - InstanceConnectionInfo one = new InstanceConnectionInfo(address1, 19871); - InstanceConnectionInfo two = new InstanceConnectionInfo(address2, 19871); - InstanceConnectionInfo three = new InstanceConnectionInfo(address3, 10871); - InstanceConnectionInfo four = new InstanceConnectionInfo(address1, 19871); + TaskManagerLocation one = new TaskManagerLocation(resourceID1, address1, 19871); + TaskManagerLocation two = new TaskManagerLocation(resourceID2, address2, 19871); + TaskManagerLocation three = new TaskManagerLocation(resourceID3, address3, 10871); + TaskManagerLocation four = new TaskManagerLocation(resourceID1, address1, 19871); assertTrue(one.equals(four)); assertTrue(!one.equals(two)); @@ -96,24 +101,20 @@ public void testSerialization() { try { // without resolved hostname { - InstanceConnectionInfo original = new InstanceConnectionInfo(InetAddress.getByName("1.2.3.4"), 8888); - - InstanceConnectionInfo copy = InstantiationUtil.createCopyWritable(original); - assertEquals(original, copy); - - InstanceConnectionInfo serCopy = InstantiationUtil.clone(original); + TaskManagerLocation original = new TaskManagerLocation( + ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888); + + TaskManagerLocation serCopy = InstantiationUtil.clone(original); assertEquals(original, serCopy); } // with resolved hostname { - InstanceConnectionInfo original = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871); + TaskManagerLocation original = new TaskManagerLocation( + ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871); original.getFQDNHostname(); - - InstanceConnectionInfo copy = InstantiationUtil.createCopyWritable(original); - assertEquals(original, copy); - - InstanceConnectionInfo serCopy = InstantiationUtil.clone(original); + + TaskManagerLocation serCopy = InstantiationUtil.clone(original); assertEquals(original, serCopy); } } @@ -126,10 +127,10 @@ public void testSerialization() { @Test public void testGetFQDNHostname() { try { - InstanceConnectionInfo info1 = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871); + TaskManagerLocation info1 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871); assertNotNull(info1.getFQDNHostname()); - InstanceConnectionInfo info2 = new InstanceConnectionInfo(InetAddress.getByName("1.2.3.4"), 8888); + TaskManagerLocation info2 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888); assertNotNull(info2.getFQDNHostname()); } catch (Exception e) { @@ -146,7 +147,7 @@ public void testGetHostname0() { when(address.getHostName()).thenReturn("worker2.cluster.mycompany.com"); when(address.getHostAddress()).thenReturn("127.0.0.1"); - final InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871); + final TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871); Assert.assertEquals("worker2", info.getHostname()); } catch (Exception e) { @@ -163,7 +164,7 @@ public void testGetHostname1() { when(address.getHostName()).thenReturn("worker10"); when(address.getHostAddress()).thenReturn("127.0.0.1"); - InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871); + TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871); Assert.assertEquals("worker10", info.getHostname()); } catch (Exception e) { @@ -184,7 +185,7 @@ public void testGetHostname2() { when(address.getHostAddress()).thenReturn("192.168.254.254"); when(address.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, (byte) 254, (byte) 254} ); - InstanceConnectionInfo info = new InstanceConnectionInfo(address, 54152); + TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 54152); assertNotNull(info.getFQDNHostname()); assertTrue(info.getFQDNHostname().equals(addressString)); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala index 7feb9493d27a4..f9c9b63ca6682 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala @@ -30,6 +30,7 @@ import org.apache.flink.runtime.instance._ import org.apache.flink.runtime.jobmanager.JobManagerRegistrationTest.PlainForwardingActor import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered, RegisterTaskManager} +import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testutils.TestingResourceManager import org.apache.flink.runtime.util.LeaderRetrievalUtils @@ -63,8 +64,11 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { val tm1 = _system.actorOf(Props(new PlainForwardingActor(testActor))) val tm2 = _system.actorOf(Props(new PlainForwardingActor(testActor))) - val connectionInfo1 = new InstanceConnectionInfo(InetAddress.getLocalHost, 10000) - val connectionInfo2 = new InstanceConnectionInfo(InetAddress.getLocalHost, 10001) + val resourceId1 = ResourceID.generate() + val resourceId2 = ResourceID.generate() + + val connectionInfo1 = new TaskManagerLocation(resourceId1, InetAddress.getLocalHost, 10000) + val connectionInfo2 = new TaskManagerLocation(resourceId2, InetAddress.getLocalHost, 10001) val hardwareDescription = HardwareDescription.extractFromSystem(10) @@ -75,7 +79,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { within(10 seconds) { jm.tell( RegisterTaskManager( - ResourceID.generate(), + resourceId1, connectionInfo1, hardwareDescription, 1), @@ -92,7 +96,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { within(10 seconds) { jm.tell( RegisterTaskManager( - ResourceID.generate(), + resourceId2, connectionInfo2, hardwareDescription, 1), @@ -118,7 +122,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { val selfGateway = new AkkaActorGateway(testActor, null) val resourceID = ResourceID.generate() - val connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost,1) + val connectionInfo = new TaskManagerLocation(resourceID, InetAddress.getLocalHost, 1) val hardwareDescription = HardwareDescription.extractFromSystem(10) within(20 seconds) { diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index 73ab7ebf39c1c..f9f294b91d017 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -19,12 +19,11 @@ package org.apache.flink.yarn import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManagerConfiguration} import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike /** [[YarnTaskManager]] implementation which mixes in the [[TestingTaskManagerLike]] mixin. @@ -43,14 +42,14 @@ import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike * JobManager */ class TestingYarnTaskManager( - config: TaskManagerConfiguration, - resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) + config: TaskManagerConfiguration, + resourceID: ResourceID, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) extends YarnTaskManager( config, resourceID, @@ -65,6 +64,7 @@ class TestingYarnTaskManager( object YarnTaskManager { /** Entry point (main method) to run the TaskManager on YARN. + * * @param args The command line arguments. */ def main(args: Array[String]): Unit = { diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 8b39f522929a3..3566ab83f2ac1 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -20,12 +20,12 @@ package org.apache.flink.yarn import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo +import org.apache.flink.runtime.instance.TaskManagerInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManagerConfiguration, TaskManager} +import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManagerConfiguration, TaskManager} import org.apache.flink.runtime.util.ProcessShutDownThread import scala.concurrent.duration._ @@ -34,14 +34,14 @@ import scala.concurrent.duration._ * messages. */ class YarnTaskManager( - config: TaskManagerConfiguration, - resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) + config: TaskManagerConfiguration, + resourceID: ResourceID, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) extends TaskManager( config, resourceID, From 02ad27e065e3f32b4503fac6ac5a3138a3b607e1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 30 Aug 2016 20:34:20 +0200 Subject: [PATCH 2/4] [FLINK-4490] [distributed coordination] (part 2) Make slots independent of 'Instance'. To allow for a future dynamic slot allocation and release model, the slots should not depend on 'Instance'. In this change, the Slots hold most of the necessary information directly (location, gateway) and the interact with the Instance only via a 'SlotOwner' interface. --- .../clusterframework/MesosTaskManager.scala | 5 +- .../InputChannelDeploymentDescriptor.java | 14 +- .../runtime/executiongraph/Execution.java | 50 +++--- .../executiongraph/ExecutionVertex.java | 17 +-- .../runtime/instance/HardwareDescription.java | 28 ++-- .../flink/runtime/instance/Instance.java | 22 +-- .../flink/runtime/instance/SharedSlot.java | 45 ++++-- .../flink/runtime/instance/SimpleSlot.java | 35 +++-- .../apache/flink/runtime/instance/Slot.java | 103 ++++++++++--- .../instance/SlotSharingGroupAssignment.java | 132 +++++++--------- .../scheduler/CoLocationConstraint.java | 48 +++--- .../jobmanager/scheduler/Scheduler.java | 74 +++++---- .../scheduler/SlotAllocationFuture.java | 116 ++++++++++---- .../runtime/jobmanager/slots/SlotOwner.java | 29 ++++ .../taskmanager/TaskManagerLocation.java | 2 +- .../flink/runtime/jobmanager/JobManager.scala | 2 +- .../testingUtils/TestingJobManagerLike.scala | 2 +- .../ExecutionGraphMetricsTest.java | 4 +- .../VertexLocationConstraintTest.java | 52 +++---- .../runtime/instance/SharedSlotsTest.java | 67 ++++---- .../ScheduleWithCoLocationHintTest.java | 144 ++++++++++-------- .../scheduler/SchedulerIsolatedTasksTest.java | 52 ++++--- .../scheduler/SchedulerSlotSharingTest.java | 102 +++++++------ .../scheduler/SchedulerTestUtils.java | 29 +++- .../scheduler/SlotAllocationFutureTest.java | 51 +++++-- .../apache/flink/yarn/YarnTaskManager.scala | 12 +- 26 files changed, 736 insertions(+), 501 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index d8b67755856e7..d07ea7be3601a 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -19,12 +19,11 @@ package org.apache.flink.mesos.runtime.clusterframework import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} /** An extension of the TaskManager that listens for additional Mesos-related * messages. @@ -32,7 +31,7 @@ import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfigurati class MesosTaskManager( config: TaskManagerConfiguration, resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, + connectionInfo: TaskManagerLocation, memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java index e00a48058965d..327238dff9faf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java @@ -18,16 +18,18 @@ 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.ExecutionEdge; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.Instance; 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.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,6 +89,7 @@ public String toString() { public static InputChannelDeploymentDescriptor[] fromEdges( ExecutionEdge[] edges, SimpleSlot consumerSlot) { + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); final InputChannelDeploymentDescriptor[] icdd = new InputChannelDeploymentDescriptor[edges.length]; // Each edge is connected to a different result partition @@ -104,16 +107,17 @@ public static InputChannelDeploymentDescriptor[] fromEdges( (producerState == ExecutionState.RUNNING || producerState == ExecutionState.FINISHED)) { - final Instance partitionInstance = producerSlot.getInstance(); + final TaskManagerLocation partitionTaskManagerLocation = producerSlot.getTaskManagerLocation(); + final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - if (partitionInstance.equals(consumerSlot.getInstance())) { - // Consuming task is deployed to the same instance as the partition => local + if (partitionTaskManager.equals(consumerTaskManager)) { + // Consuming task is deployed to the same TaskManager as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } else { // Different instances => remote final ConnectionID connectionId = new ConnectionID( - partitionInstance.getInstanceConnectionInfo(), + partitionTaskManagerLocation, consumedPartition.getIntermediateResult().getConnectionIndex()); partitionLocation = ResultPartitionLocation.createRemote(connectionId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 16f5d34b85bce..1def896f19370 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -25,12 +25,12 @@ import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor; 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.instance.Instance; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; @@ -356,7 +356,7 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { throw new JobException("Could not assign the ExecutionVertex to the slot " + slot); } this.assignedResource = slot; - this.assignedResourceLocation = slot.getInstance().getInstanceConnectionInfo(); + this.assignedResourceLocation = slot.getTaskManagerLocation(); // race double check, did we fail/cancel and do we need to release the slot? if (this.state != DEPLOYING) { @@ -366,7 +366,7 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { if (LOG.isInfoEnabled()) { LOG.info(String.format("Deploying %s (attempt #%d) to %s", vertex.getSimpleName(), - attemptNumber, slot.getInstance().getInstanceConnectionInfo().getHostname())); + attemptNumber, assignedResourceLocation.getHostname())); } final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor( @@ -377,9 +377,8 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { // register this execution at the execution graph, to receive call backs vertex.getExecutionGraph().registerExecution(this); - - final Instance instance = slot.getInstance(); - final ActorGateway gateway = instance.getActorGateway(); + + final ActorGateway gateway = slot.getTaskManagerActorGateway(); final Future deployAction = gateway.ask(new SubmitTask(deployment), timeout); @@ -392,7 +391,7 @@ public void onComplete(Throwable failure, Object success) throws Throwable { String taskname = deployment.getTaskInfo().getTaskNameWithSubtasks() + " (" + attemptId + ')'; markFailed(new Exception( - "Cannot deploy task " + taskname + " - TaskManager (" + instance + "Cannot deploy task " + taskname + " - TaskManager (" + assignedResourceLocation + ") not responding after a timeout of " + timeout, failure)); } else { @@ -421,7 +420,7 @@ public void stop() { final SimpleSlot slot = this.assignedResource; if (slot != null) { - final ActorGateway gateway = slot.getInstance().getActorGateway(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); Future stopResult = gateway.retry( new StopTask(attemptId), @@ -574,24 +573,25 @@ public Boolean call() throws Exception { continue; } - final Instance consumerInstance = consumerSlot.getInstance(); - - final ResultPartitionID partitionId = new ResultPartitionID( - partition.getPartitionId(), attemptId); + final TaskManagerLocation partitionTaskManagerLocation = partition.getProducer() + .getCurrentAssignedResource().getTaskManagerLocation(); + final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); + + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); - final Instance partitionInstance = partition.getProducer() - .getCurrentAssignedResource().getInstance(); + final ResultPartitionID partitionId = new ResultPartitionID(partition.getPartitionId(), attemptId); + final ResultPartitionLocation partitionLocation; - if (consumerInstance.equals(partitionInstance)) { + if (consumerTaskManager.equals(partitionTaskManager)) { // Consuming task is deployed to the same instance as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } else { // Different instances => remote final ConnectionID connectionId = new ConnectionID( - partitionInstance.getInstanceConnectionInfo(), + partitionTaskManagerLocation, partition.getIntermediateResult().getConnectionIndex()); partitionLocation = ResultPartitionLocation.createRemote(connectionId); @@ -900,7 +900,7 @@ private void sendCancelRpcCall() { if (slot != null) { - final ActorGateway gateway = slot.getInstance().getActorGateway(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); Future cancelResult = gateway.retry( new CancelTask(attemptId), @@ -930,14 +930,10 @@ private void sendFailIntermediateResultPartitionsRpcCall() { final SimpleSlot slot = this.assignedResource; if (slot != null) { - final Instance instance = slot.getInstance(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); - if (instance.isAlive()) { - final ActorGateway gateway = instance.getActorGateway(); - - // TODO For some tests this could be a problem when querying too early if all resources were released - gateway.tell(new FailIntermediateResultPartitions(attemptId)); - } + // TODO For some tests this could be a problem when querying too early if all resources were released + gateway.tell(new FailIntermediateResultPartitions(attemptId)); } } @@ -952,15 +948,15 @@ private void sendUpdatePartitionInfoRpcCall( final UpdatePartitionInfo updatePartitionInfo) { if (consumerSlot != null) { - final Instance instance = consumerSlot.getInstance(); - final ActorGateway gateway = instance.getActorGateway(); + final ActorGateway gateway = consumerSlot.getTaskManagerActorGateway(); + final TaskManagerLocation taskManagerLocation = consumerSlot.getTaskManagerLocation(); Future futureUpdate = gateway.ask(updatePartitionInfo, timeout); futureUpdate.onFailure(new OnFailure() { @Override public void onFailure(Throwable failure) throws Throwable { - fail(new IllegalStateException("Update task on instance " + instance + + fail(new IllegalStateException("Update task on TaskManager " + taskManagerLocation + " failed due to:", failure)); } }, executionContext); 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 f74a3491e93ce..8da3dc11a001d 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 @@ -27,7 +27,6 @@ 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.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; @@ -96,7 +95,7 @@ public class ExecutionVertex { private volatile Execution currentExecution; // this field must never be null - private volatile List locationConstraintInstances; + private volatile List locationConstraintInstances; private volatile boolean scheduleLocalOnly; @@ -346,7 +345,7 @@ else if (numSources < parallelism) { } } - public void setLocationConstraintHosts(List instances) { + public void setLocationConstraintHosts(List instances) { this.locationConstraintInstances = instances; } @@ -370,9 +369,9 @@ public boolean isScheduleLocalOnly() { * * @return The preferred locations for this vertex execution, or null, if there is no preference. */ - public Iterable getPreferredLocations() { + public Iterable getPreferredLocations() { // if we have hard location constraints, use those - List constraintInstances = this.locationConstraintInstances; + List constraintInstances = this.locationConstraintInstances; if (constraintInstances != null && !constraintInstances.isEmpty()) { return constraintInstances; } @@ -382,8 +381,8 @@ public Iterable getPreferredLocations() { return Collections.emptySet(); } else { - Set locations = new HashSet(); - Set inputLocations = new HashSet(); + Set locations = new HashSet<>(); + Set inputLocations = new HashSet<>(); // go over all inputs for (int i = 0; i < inputEdges.length; i++) { @@ -396,7 +395,7 @@ public Iterable getPreferredLocations() { SimpleSlot sourceSlot = sources[k].getSource().getProducer().getCurrentAssignedResource(); if (sourceSlot != null) { // add input location - inputLocations.add(sourceSlot.getInstance()); + inputLocations.add(sourceSlot.getTaskManagerLocation()); // inputs which have too many distinct sources are not considered if (inputLocations.size() > MAX_DISTINCT_LOCATIONS_TO_CONSIDER) { inputLocations.clear(); @@ -489,7 +488,7 @@ public boolean sendMessageToCurrentExecution( // send only if we actually have a target if (slot != null) { - ActorGateway gateway = slot.getInstance().getActorGateway(); + ActorGateway gateway = slot.getTaskManagerActorGateway(); if (gateway != null) { if (sender == null) { gateway.tell(message); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java index bfcc1e5c141a4..9c1c5b78a45f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java @@ -30,22 +30,16 @@ public final class HardwareDescription implements Serializable { private static final long serialVersionUID = 3380016608300325361L; /** The number of CPU cores available to the JVM on the compute node. */ - private int numberOfCPUCores; + private final int numberOfCPUCores; /** The size of physical memory in bytes available on the compute node. */ - private long sizeOfPhysicalMemory; + private final long sizeOfPhysicalMemory; /** The size of the JVM heap memory */ - private long sizeOfJvmHeap; - - /** The size of the memory managed by the system for caching, hashing, sorting, ... */ - private long sizeOfManagedMemory; + private final long sizeOfJvmHeap; - - /** - * Public default constructor used for serialization process. - */ - public HardwareDescription() {} + /** The size of the memory managed by the system for caching, hashing, sorting, ... */ + private final long sizeOfManagedMemory; /** * Constructs a new hardware description object. @@ -88,7 +82,7 @@ public long getSizeOfPhysicalMemory() { public long getSizeOfJvmHeap() { return this.sizeOfJvmHeap; } - + /** * Returns the size of the memory managed by the system for caching, hashing, sorting, ... * @@ -97,26 +91,26 @@ public long getSizeOfJvmHeap() { public long getSizeOfManagedMemory() { return this.sizeOfManagedMemory; } - + // -------------------------------------------------------------------------------------------- // Utils // -------------------------------------------------------------------------------------------- - + @Override public String toString() { return String.format("cores=%d, physMem=%d, heap=%d, managed=%d", numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, sizeOfManagedMemory); } - + // -------------------------------------------------------------------------------------------- // Factory // -------------------------------------------------------------------------------------------- - + public static HardwareDescription extractFromSystem(long managedMemory) { final int numberOfCPUCores = Hardware.getNumberCPUCores(); final long sizeOfJvmHeap = Runtime.getRuntime().maxMemory(); final long sizeOfPhysicalMemory = Hardware.getSizeOfPhysicalMemory(); - + return new HardwareDescription(numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, managedMemory); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index 598b32bbc89ac..fe46895c4500b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -28,15 +28,20 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * An instance represents a {@link org.apache.flink.runtime.taskmanager.TaskManager} * registered at a JobManager and ready to receive work. */ -public class Instance { +public class Instance implements SlotOwner { private final static Logger LOG = LoggerFactory.getLogger(Instance.class); @@ -241,7 +246,7 @@ public SimpleSlot allocateSimpleSlot(JobID jobID) throws InstanceDiedException { return null; } else { - SimpleSlot slot = new SimpleSlot(jobID, this, nextSlot); + SimpleSlot slot = new SimpleSlot(jobID, this, connectionInfo, nextSlot, actorGateway); allocatedSlots.add(slot); return slot; } @@ -278,7 +283,8 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha return null; } else { - SharedSlot slot = new SharedSlot(jobID, this, nextSlot, sharingGroupAssignment); + SharedSlot slot = new SharedSlot( + jobID, this, connectionInfo, nextSlot, actorGateway, sharingGroupAssignment); allocatedSlots.add(slot); return slot; } @@ -295,13 +301,11 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha * @param slot The slot to return. * @return True, if the slot was returned, false if not. */ + @Override public boolean returnAllocatedSlot(Slot slot) { - if (slot == null || slot.getInstance() != this) { - throw new IllegalArgumentException("Slot is null or belongs to the wrong TaskManager."); - } - if (slot.isAlive()) { - throw new IllegalArgumentException("Slot is still alive"); - } + checkNotNull(slot); + checkArgument(!slot.isAlive(), "slot is still alive"); + checkArgument(slot.getOwner() == this, "slot belongs to the wrong TaskManager."); if (slot.markReleased()) { LOG.debug("Return allocated slot {}.", slot); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java index ef6291069ced2..7f056042b9098 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java @@ -18,13 +18,18 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.api.common.JobID; +import javax.annotation.Nullable; import java.util.ConcurrentModificationException; import java.util.HashSet; import java.util.Set; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * This class represents a shared slot. A shared slot can have multiple * {@link SimpleSlot} instances within itself. This allows to @@ -35,7 +40,7 @@ *

IMPORTANT: This class contains no synchronization. Thus, the caller has to guarantee proper * synchronization. In the current implementation, all concurrently modifying operations are * passed through a {@link SlotSharingGroupAssignment} object which is responsible for - * synchronization.

+ * synchronization. */ public class SharedSlot extends Slot { @@ -51,12 +56,18 @@ public class SharedSlot extends Slot { * This constructor is used to create a slot directly from an instance. * * @param jobID The ID of the job that the slot is created for. - * @param instance The instance that holds the slot. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param assignmentGroup The assignment group that this shared slot belongs to. */ - public SharedSlot(JobID jobID, Instance instance, int slotNumber, SlotSharingGroupAssignment assignmentGroup) { - this(jobID, instance, slotNumber, assignmentGroup, null, null); + public SharedSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + SlotSharingGroupAssignment assignmentGroup) { + + this(jobID, owner, location, slotNumber, taskManagerActorGateway, assignmentGroup, null, null); } /** @@ -64,15 +75,23 @@ public SharedSlot(JobID jobID, Instance instance, int slotNumber, SlotSharingGro * to the given task group. * * @param jobID The ID of the job that the slot is created for. - * @param instance The instance that holds the slot. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param assignmentGroup The assignment group that this shared slot belongs to. + * @param parent The parent slot of this slot. + * @param groupId The assignment group of this slot. */ - public SharedSlot(JobID jobID, Instance instance, int slotNumber, - SlotSharingGroupAssignment assignmentGroup, SharedSlot parent, AbstractID groupId) { - super(jobID, instance, slotNumber, parent, groupId); + public SharedSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + SlotSharingGroupAssignment assignmentGroup, + @Nullable SharedSlot parent, @Nullable AbstractID groupId) { + + super(jobID, owner, location, slotNumber, taskManagerActorGateway, parent, groupId); - this.assignmentGroup = assignmentGroup; + this.assignmentGroup = checkNotNull(assignmentGroup); this.subSlots = new HashSet(); } @@ -148,7 +167,9 @@ Set getSubSlots() { */ SimpleSlot allocateSubSlot(AbstractID groupId) { if (isAlive()) { - SimpleSlot slot = new SimpleSlot(getJobID(), getInstance(), subSlots.size(), this, groupId); + SimpleSlot slot = new SimpleSlot( + getJobID(), getOwner(), getTaskManagerLocation(), subSlots.size(), + getTaskManagerActorGateway(), this, groupId); subSlots.add(slot); return slot; } @@ -168,7 +189,9 @@ SimpleSlot allocateSubSlot(AbstractID groupId) { */ SharedSlot allocateSharedSlot(AbstractID groupId){ if (isAlive()) { - SharedSlot slot = new SharedSlot(getJobID(), getInstance(), subSlots.size(), assignmentGroup, this, groupId); + SharedSlot slot = new SharedSlot( + getJobID(), getOwner(), getTaskManagerLocation(), subSlots.size(), + getTaskManagerActorGateway(), assignmentGroup, this, groupId); subSlots.add(slot); return slot; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java index dbe961ae1d706..615138f42698b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java @@ -21,15 +21,18 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; +import javax.annotation.Nullable; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; /** * A SimpleSlot represents a single slot on a TaskManager instance, or a slot within a shared slot. * *

If this slot is part of a {@link SharedSlot}, then the parent attribute will point to that shared slot. - * If not, then the parent attribute is null.

+ * If not, then the parent attribute is null. */ public class SimpleSlot extends Slot { @@ -43,18 +46,22 @@ public class SimpleSlot extends Slot { private volatile Execution executedTask; /** The locality attached to the slot, defining whether the slot was allocated at the desired location. */ - private Locality locality = Locality.UNCONSTRAINED; + private volatile Locality locality = Locality.UNCONSTRAINED; /** * Creates a new simple slot that stands alone and does not belong to shared slot. * * @param jobID The ID of the job that the slot is allocated for. - * @param instance The instance that the slot belongs to. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the task slot on the instance. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager of this slot */ - public SimpleSlot(JobID jobID, Instance instance, int slotNumber) { - super(jobID, instance, slotNumber, null, null); + public SimpleSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway) { + this(jobID, owner, location, slotNumber, taskManagerActorGateway, null, null); } /** @@ -62,13 +69,18 @@ public SimpleSlot(JobID jobID, Instance instance, int slotNumber) { * is identified by the given ID.. * * @param jobID The ID of the job that the slot is allocated for. - * @param instance The instance that the slot belongs to. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the simple slot in its parent shared slot. * @param parent The parent shared slot. * @param groupID The ID that identifies the group that the slot belongs to. */ - public SimpleSlot(JobID jobID, Instance instance, int slotNumber, SharedSlot parent, AbstractID groupID) { - super(jobID, instance, slotNumber, parent, groupID); + public SimpleSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + @Nullable SharedSlot parent, @Nullable AbstractID groupID) { + + super(jobID, owner, location, slotNumber, taskManagerActorGateway, parent, groupID); } // ------------------------------------------------------------------------ @@ -142,15 +154,12 @@ public void setLocality(Locality locality) { @Override public void releaseSlot() { - if (!isCanceled()) { // kill all tasks currently running in this slot Execution exec = this.executedTask; if (exec != null && !exec.isFinished()) { - exec.fail(new Exception( - "The slot in which the task was executed has been released. Probably loss of TaskManager " - + getInstance())); + exec.fail(new Exception("TaskManager was lost/killed: " + getTaskManagerLocation())); } // release directly (if we are directly allocated), @@ -158,7 +167,7 @@ public void releaseSlot() { if (getParent() == null) { // we have to give back the slot to the owning instance if (markCancelled()) { - getInstance().returnAllocatedSlot(this); + getOwner().returnAllocatedSlot(this); } } else { // we have to ask our parent to dispose us diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java index 341ef95481681..451a9ecbed595 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java @@ -18,11 +18,18 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.api.common.JobID; +import javax.annotation.Nullable; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Base class for task slots. TaskManagers offer one or more task slots, which define a slice of * their resources. @@ -30,7 +37,7 @@ *

In the simplest case, a slot holds a single task ({@link SimpleSlot}). In the more complex * case, a slot is shared ({@link SharedSlot}) and contains a set of tasks. Shared slots may contain * other shared slots which in turn can hold simple slots. That way, a shared slot may define a tree - * of slots that belong to it.

+ * of slots that belong to it. */ public abstract class Slot { @@ -52,15 +59,23 @@ public abstract class Slot { /** The ID of the job this slice belongs to. */ private final JobID jobID; - /** The id of the group that this slot is allocated to. May be null. */ - private final AbstractID groupID; + /** The location information of the TaskManager to which this slot belongs */ + private final TaskManagerLocation taskManagerLocation; + + /** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */ + private final ActorGateway taskManagerActorGateway; - /** The instance on which the slot is allocated */ - private final Instance instance; + /** The owner of this slot - the slot was taken from that owner and must be disposed to it */ + private final SlotOwner owner; /** The parent of this slot in the hierarchy, or null, if this is the parent */ + @Nullable private final SharedSlot parent; + /** The id of the group that this slot is allocated to. May be null. */ + @Nullable + private final AbstractID groupID; + /** The number of the slot on which the task is deployed */ private final int slotNumber; @@ -71,23 +86,28 @@ public abstract class Slot { * Base constructor for slots. * * @param jobID The ID of the job that this slot is allocated for. - * @param instance The instance from which this slot is allocated. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of this slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param parent The parent slot that contains this slot. May be null, if this slot is the root. * @param groupID The ID that identifies the task group for which this slot is allocated. May be null * if the slot does not belong to any task group. */ - protected Slot(JobID jobID, Instance instance, int slotNumber, SharedSlot parent, AbstractID groupID) { - if (jobID == null || instance == null || slotNumber < 0) { - throw new IllegalArgumentException(); - } - - this.jobID = jobID; - this.instance = instance; + protected Slot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + @Nullable SharedSlot parent, @Nullable AbstractID groupID) { + + checkArgument(slotNumber >= 0); + + this.jobID = checkNotNull(jobID); + this.taskManagerLocation = checkNotNull(location); + this.owner = checkNotNull(owner); + this.taskManagerActorGateway = checkNotNull(taskManagerActorGateway); + this.parent = parent; // may be null + this.groupID = groupID; // may be null this.slotNumber = slotNumber; - this.parent = parent; - this.groupID = groupID; - } // -------------------------------------------------------------------------------------------- @@ -101,12 +121,42 @@ public JobID getJobID() { } /** - * Gets the instance from which the slot was allocated. + * Gets the ID of the TaskManager that offers this slot. + * + * @return The ID of the TaskManager that offers this slot + */ + public ResourceID getTaskManagerID() { + return taskManagerLocation.getResourceID(); + } + + /** + * Gets the location info of the TaskManager that offers this slot. * - * @return The instance from which the slot was allocated. + * @return The location info of the TaskManager that offers this slot */ - public Instance getInstance() { - return instance; + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + + /** + * Gets the actor gateway that can be used to send messages to the TaskManager. + * + *

This method should be removed once the new interface-based RPC abstraction is in place + * + * @return The actor gateway that can be used to send messages to the TaskManager. + */ + public ActorGateway getTaskManagerActorGateway() { + return taskManagerActorGateway; + } + + /** + * Gets the owner of this slot. The owner is the component that the slot was created from + * and to which it needs to be returned after the executed tasks are done. + * + * @return The owner of this slot. + */ + public SlotOwner getOwner() { + return owner; } /** @@ -149,6 +199,7 @@ public int getRootSlotNumber() { * * @return The ID identifying the logical group of slots. */ + @Nullable public AbstractID getGroupID() { return groupID; } @@ -158,10 +209,18 @@ public AbstractID getGroupID() { * * @return The parent slot, or null, if no this slot has no parent. */ + @Nullable public SharedSlot getParent() { return parent; } + /** + * Gets the root slot of the tree containing this slot. If this slot is the root, + * the method returns this slot directly, otherwise it recursively goes to the parent until + * it reaches the root. + * + * @return The root slot of the tree containing this slot + */ public Slot getRoot() { if (parent == null) { return this; @@ -244,11 +303,11 @@ final boolean markReleased() { @Override public String toString() { - return hierarchy() + " - " + instance + " - " + getStateName(status); + return hierarchy() + " - " + taskManagerLocation + " - " + getStateName(status); } protected String hierarchy() { - return (getParent() != null ? getParent().hierarchy() : "") + "(" + slotNumber + ")"; + return (getParent() != null ? getParent().hierarchy() : "") + '(' + slotNumber + ')'; } private static String getStateName(int state) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java index 7d666fe09dbf5..346cc77b46937 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java @@ -28,13 +28,15 @@ import java.util.Map; import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,9 +95,8 @@ public class SlotSharingGroupAssignment { /** All slots currently allocated to this sharing group */ private final Set allSlots = new LinkedHashSet(); - /** The slots available per vertex type (jid), keyed by instance, to make them locatable */ - private final Map>> availableSlotsPerJid = - new LinkedHashMap>>(); + /** The slots available per vertex type (JobVertexId), keyed by TaskManager, to make them locatable */ + private final Map>> availableSlotsPerJid = new LinkedHashMap<>(); // -------------------------------------------------------------------------------------------- @@ -122,7 +123,7 @@ public int getNumberOfSlots() { */ public int getNumberOfAvailableSlotsForGroup(AbstractID groupId) { synchronized (lock) { - Map> available = availableSlotsPerJid.get(groupId); + Map> available = availableSlotsPerJid.get(groupId); if (available != null) { Set set = new HashSet(); @@ -148,37 +149,25 @@ public int getNumberOfAvailableSlotsForGroup(AbstractID groupId) { // Slot allocation // ------------------------------------------------------------------------ - /** - * - * @param sharedSlot - * @param locality - * @param groupId - * @return - */ public SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, JobVertexID groupId) { return addSharedSlotAndAllocateSubSlot(sharedSlot, locality, groupId, null); } - /** - * - * @param sharedSlot - * @param locality - * @param constraint - * @return - */ - public SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, - CoLocationConstraint constraint) { + public SimpleSlot addSharedSlotAndAllocateSubSlot( + SharedSlot sharedSlot, Locality locality, CoLocationConstraint constraint) + { return addSharedSlotAndAllocateSubSlot(sharedSlot, locality, null, constraint); } - - private SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, - JobVertexID groupId, CoLocationConstraint constraint) { + + private SimpleSlot addSharedSlotAndAllocateSubSlot( + SharedSlot sharedSlot, Locality locality, JobVertexID groupId, CoLocationConstraint constraint) { + // sanity checks if (!sharedSlot.isRootAndEmpty()) { throw new IllegalArgumentException("The given slot is not an empty root slot."); } - - final Instance location = sharedSlot.getInstance(); + + final ResourceID location = sharedSlot.getTaskManagerID(); synchronized (lock) { // early out in case that the slot died (instance disappeared) @@ -244,20 +233,20 @@ private SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locali // can place a task into this slot. boolean entryForNewJidExists = false; - for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { + for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { // there is already an entry for this groupID if (entry.getKey().equals(groupIdForMap)) { entryForNewJidExists = true; continue; } - Map> available = entry.getValue(); + Map> available = entry.getValue(); putIntoMultiMap(available, location, sharedSlot); } // make sure an empty entry exists for this group, if no other entry exists if (!entryForNewJidExists) { - availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap>()); + availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap>()); } return subSlot; @@ -287,18 +276,15 @@ public SimpleSlot getSlotForTask(ExecutionVertex vertex) { /** * - * @param vertexID - * @param locationPreferences - * @return */ - SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable locationPreferences) { + SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable locationPreferences) { synchronized (lock) { - Pair p = getSlotForTaskInternal(vertexID, locationPreferences, false); + Tuple2 p = getSlotForTaskInternal(vertexID, locationPreferences, false); if (p != null) { - SharedSlot ss = p.getLeft(); + SharedSlot ss = p.f0; SimpleSlot slot = ss.allocateSubSlot(vertexID); - slot.setLocality(p.getRight()); + slot.setLocality(p.f1); return slot; } else { @@ -330,7 +316,7 @@ public SimpleSlot getSlotForTask(ExecutionVertex vertex, CoLocationConstraint co return getSlotForTask(constraint, vertex.getPreferredLocations()); } - SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable locationPreferences) { + SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable locationPreferences) { synchronized (lock) { if (constraint.isAssignedAndAlive()) { // the shared slot of the co-location group is initialized and set we allocate a sub-slot @@ -346,15 +332,16 @@ else if (constraint.isAssigned()) { if (previous == null) { throw new IllegalStateException("Bug: Found assigned co-location constraint without a slot."); } - - Instance location = previous.getInstance(); - Pair p = getSlotForTaskInternal(constraint.getGroupId(), - Collections.singleton(location), true); + + TaskManagerLocation location = previous.getTaskManagerLocation(); + Tuple2 p = getSlotForTaskInternal( + constraint.getGroupId(), Collections.singleton(location), true); + if (p == null) { return null; } else { - SharedSlot newSharedSlot = p.getLeft(); + SharedSlot newSharedSlot = p.f0; // allocate the co-location group slot inside the shared slot SharedSlot constraintGroupSlot = newSharedSlot.allocateSharedSlot(constraint.getGroupId()); @@ -377,15 +364,15 @@ else if (constraint.isAssigned()) { // the location constraint has not been associated with a shared slot, yet. // grab a new slot and initialize the constraint with that one. // preferred locations are defined by the vertex - Pair p = + Tuple2 p = getSlotForTaskInternal(constraint.getGroupId(), locationPreferences, false); if (p == null) { // could not get a shared slot for this co-location-group return null; } else { - final SharedSlot availableShared = p.getLeft(); - final Locality l = p.getRight(); + final SharedSlot availableShared = p.f0; + final Locality l = p.f1; // allocate the co-location group slot inside the shared slot SharedSlot constraintGroupSlot = availableShared.allocateSharedSlot(constraint.getGroupId()); @@ -405,9 +392,8 @@ else if (constraint.isAssigned()) { } - private Pair getSlotForTaskInternal(AbstractID groupId, - Iterable preferredLocations, - boolean localOnly) + private Tuple2 getSlotForTaskInternal( + AbstractID groupId, Iterable preferredLocations, boolean localOnly) { // check if there is anything at all in this group assignment if (allSlots.isEmpty()) { @@ -415,15 +401,15 @@ private Pair getSlotForTaskInternal(AbstractID groupId, } // get the available slots for the group - Map> slotsForGroup = availableSlotsPerJid.get(groupId); + Map> slotsForGroup = availableSlotsPerJid.get(groupId); if (slotsForGroup == null) { // we have a new group, so all slots are available - slotsForGroup = new LinkedHashMap>(); + slotsForGroup = new LinkedHashMap<>(); availableSlotsPerJid.put(groupId, slotsForGroup); for (SharedSlot availableSlot : allSlots) { - putIntoMultiMap(slotsForGroup, availableSlot.getInstance(), availableSlot); + putIntoMultiMap(slotsForGroup, availableSlot.getTaskManagerID(), availableSlot); } } else if (slotsForGroup.isEmpty()) { @@ -435,15 +421,15 @@ else if (slotsForGroup.isEmpty()) { boolean didNotGetPreferred = false; if (preferredLocations != null) { - for (Instance location : preferredLocations) { + for (TaskManagerLocation location : preferredLocations) { // set the flag that we failed a preferred location. If one will be found, // we return early anyways and skip the flag evaluation didNotGetPreferred = true; - SharedSlot slot = removeFromMultiMap(slotsForGroup, location); + SharedSlot slot = removeFromMultiMap(slotsForGroup, location.getResourceID()); if (slot != null && slot.isAlive()) { - return new ImmutablePair(slot, Locality.LOCAL); + return new Tuple2<>(slot, Locality.LOCAL); } } } @@ -459,7 +445,7 @@ else if (slotsForGroup.isEmpty()) { SharedSlot slot; while ((slot = pollFromMultiMap(slotsForGroup)) != null) { if (slot.isAlive()) { - return new ImmutablePair(slot, locality); + return new Tuple2<>(slot, locality); } } @@ -510,7 +496,7 @@ void releaseSimpleSlot(SimpleSlot simpleSlot) { // for that group again. otherwise, the slot is part of a // co-location group and nothing becomes immediately available - Map> slotsForJid = availableSlotsPerJid.get(groupID); + Map> slotsForJid = availableSlotsPerJid.get(groupID); // sanity check if (slotsForJid == null) { @@ -518,7 +504,7 @@ void releaseSimpleSlot(SimpleSlot simpleSlot) { " when available slots indicated that all slots were available."); } - putIntoMultiMap(slotsForJid, parent.getInstance(), parent); + putIntoMultiMap(slotsForJid, parent.getTaskManagerID(), parent); } } else { // the parent shared slot is now empty and can be released @@ -558,8 +544,6 @@ void releaseSharedSlot(SharedSlot sharedSlot) { /** * *

NOTE: This method must be called from within a scope that holds the lock.

- * - * @param sharedSlot */ private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) { // sanity check @@ -576,7 +560,7 @@ private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) { if (parent == null) { // root slot, return to the instance. - sharedSlot.getInstance().returnAllocatedSlot(sharedSlot); + sharedSlot.getOwner().returnAllocatedSlot(sharedSlot); // also, make sure we remove this slot from everywhere allSlots.remove(sharedSlot); @@ -592,7 +576,7 @@ else if (groupID != null) { if (parentRemaining > 0) { // the parent becomes available for the group again - Map> slotsForGroup = availableSlotsPerJid.get(groupID); + Map> slotsForGroup = availableSlotsPerJid.get(groupID); // sanity check if (slotsForGroup == null) { @@ -600,7 +584,7 @@ else if (groupID != null) { " when available slots indicated that all slots were available."); } - putIntoMultiMap(slotsForGroup, parent.getInstance(), parent); + putIntoMultiMap(slotsForGroup, parent.getTaskManagerID(), parent); } else { @@ -620,7 +604,7 @@ else if (groupID != null) { // Utilities // ------------------------------------------------------------------------ - private static void putIntoMultiMap(Map> map, Instance location, SharedSlot slot) { + private static void putIntoMultiMap(Map> map, ResourceID location, SharedSlot slot) { List slotsForInstance = map.get(location); if (slotsForInstance == null) { slotsForInstance = new ArrayList(); @@ -629,7 +613,7 @@ private static void putIntoMultiMap(Map> map, Instanc slotsForInstance.add(slot); } - private static SharedSlot removeFromMultiMap(Map> map, Instance location) { + private static SharedSlot removeFromMultiMap(Map> map, ResourceID location) { List slotsForLocation = map.get(location); if (slotsForLocation == null) { @@ -645,8 +629,8 @@ private static SharedSlot removeFromMultiMap(Map> map } } - private static SharedSlot pollFromMultiMap(Map> map) { - Iterator>> iter = map.entrySet().iterator(); + private static SharedSlot pollFromMultiMap(Map> map) { + Iterator>> iter = map.entrySet().iterator(); while (iter.hasNext()) { List slots = iter.next().getValue(); @@ -667,19 +651,19 @@ else if (slots.size() == 1) { return null; } - private static void removeSlotFromAllEntries(Map>> availableSlots, - SharedSlot slot) + private static void removeSlotFromAllEntries( + Map>> availableSlots, SharedSlot slot) { - final Instance instance = slot.getInstance(); + final ResourceID taskManagerId = slot.getTaskManagerID(); - for (Map.Entry>> entry : availableSlots.entrySet()) { - Map> map = entry.getValue(); + for (Map.Entry>> entry : availableSlots.entrySet()) { + Map> map = entry.getValue(); - List list = map.get(instance); + List list = map.get(taskManagerId); if (list != null) { list.remove(slot); if (list.isEmpty()) { - map.remove(instance); + map.remove(taskManagerId); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java index fece894c3953e..c41f7bfc3c666 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java @@ -18,34 +18,39 @@ package org.apache.flink.runtime.jobmanager.scheduler; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.util.Preconditions; import org.apache.flink.runtime.instance.SharedSlot; +import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A CoLocationConstraint manages the location of a set of tasks * (Execution Vertices). In co-location groups, the different subtasks of * different JobVertices need to be executed on the same {@link Instance}. * This is realized by creating a special shared slot that holds these tasks. * - *

This class tracks the location and the shared slot for this set of tasks.

+ *

This class tracks the location and the shared slot for this set of tasks. */ public class CoLocationConstraint { - + private final CoLocationGroup group; - + private volatile SharedSlot sharedSlot; - - private volatile boolean locationLocked; - - + + private volatile ResourceID lockedLocation; + + CoLocationConstraint(CoLocationGroup group) { Preconditions.checkNotNull(group); this.group = group; } - + // ------------------------------------------------------------------------ // Status & Properties // ------------------------------------------------------------------------ @@ -77,7 +82,7 @@ public AbstractID getGroupId() { * @return True if the location has been assigned, false otherwise. */ public boolean isAssigned() { - return locationLocked; + return lockedLocation != null; } /** @@ -89,7 +94,7 @@ public boolean isAssigned() { * false otherwise. */ public boolean isAssignedAndAlive() { - return locationLocked && sharedSlot.isAlive(); + return lockedLocation != null && sharedSlot.isAlive(); } /** @@ -100,9 +105,9 @@ public boolean isAssignedAndAlive() { * @return The instance describing the location for the tasks of this constraint. * @throws IllegalStateException Thrown if the location has not been assigned, yet. */ - public Instance getLocation() { - if (locationLocked) { - return sharedSlot.getInstance(); + public TaskManagerLocation getLocation() { + if (lockedLocation != null) { + return sharedSlot.getTaskManagerLocation(); } else { throw new IllegalStateException("Location not yet locked"); } @@ -125,18 +130,20 @@ public Instance getLocation() { * the new slot is from a different location. */ public void setSharedSlot(SharedSlot newSlot) { + checkNotNull(newSlot); + if (this.sharedSlot == null) { this.sharedSlot = newSlot; } else if (newSlot != this.sharedSlot){ - if (locationLocked && this.sharedSlot.getInstance() != newSlot.getInstance()) { + if (lockedLocation != null && lockedLocation != newSlot.getTaskManagerID()) { throw new IllegalArgumentException( "Cannot assign different location to a constraint whose location is locked."); } if (this.sharedSlot.isAlive()) { this.sharedSlot.releaseSlot(); } - + this.sharedSlot = newSlot; } } @@ -149,13 +156,10 @@ else if (newSlot != this.sharedSlot){ * or is no slot has been set, yet. */ public void lockLocation() throws IllegalStateException { - if (locationLocked) { - throw new IllegalStateException("Location is already locked"); - } - if (sharedSlot == null) { - throw new IllegalStateException("Cannot lock location without a slot."); - } - locationLocked = true; + checkState(lockedLocation == null, "Location is already locked"); + checkState(sharedSlot != null, "Cannot lock location without a slot."); + + lockedLocation = sharedSlot.getTaskManagerID(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index 963fc4c109b87..b481b55ec12b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Queue; @@ -37,6 +38,7 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.SlotSharingGroupAssignment; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.instance.SharedSlot; @@ -45,6 +47,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceDiedException; import org.apache.flink.runtime.instance.InstanceListener; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -78,7 +81,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener { private final HashMap> allInstancesByHost = new HashMap>(); /** All instances that still have available resources */ - private final Queue instancesWithAvailableResources = new SetQueue(); + private final Map instancesWithAvailableResources = new LinkedHashMap<>(); /** All tasks pending to be scheduled */ private final Queue taskQueue = new ArrayDeque(); @@ -163,7 +166,7 @@ private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throw final ExecutionVertex vertex = task.getTaskToExecute().getVertex(); - final Iterable preferredLocations = vertex.getPreferredLocations(); + final Iterable preferredLocations = vertex.getPreferredLocations(); final boolean forceExternalLocation = vertex.isScheduleLocalOnly() && preferredLocations != null && preferredLocations.iterator().hasNext(); @@ -222,7 +225,7 @@ private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throw // our location preference is either determined by the location constraint, or by the // vertex's preferred locations - final Iterable locations; + final Iterable locations; final boolean localOnly; if (constraint != null && constraint.isAssigned()) { locations = Collections.singleton(constraint.getLocation()); @@ -341,7 +344,7 @@ else if (forceExternalLocation) { * @return The instance to run the vertex on, it {@code null}, if no instance is available. */ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, - Iterable requestedLocations, + Iterable requestedLocations, boolean localOnly) { // we need potentially to loop multiple times, because there may be false positives // in the set-with-available-instances @@ -360,7 +363,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.add(instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); } if (slot != null) { @@ -396,7 +399,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, * @return A sub-slot for the given vertex, or {@code null}, if no slot is available. */ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, - Iterable requestedLocations, + Iterable requestedLocations, SlotSharingGroupAssignment groupAssignment, CoLocationConstraint constraint, boolean localOnly) @@ -422,7 +425,7 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.add(instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); } if (sharedSlot != null) { @@ -460,13 +463,13 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, * no locality preference exists. * @param localOnly Flag to indicate whether only one of the exact local instances can be chosen. */ - private Pair findInstance(Iterable requestedLocations, boolean localOnly){ + private Pair findInstance(Iterable requestedLocations, boolean localOnly) { // drain the queue of newly available instances while (this.newlyAvailableInstances.size() > 0) { Instance queuedInstance = this.newlyAvailableInstances.poll(); if (queuedInstance != null) { - this.instancesWithAvailableResources.add(queuedInstance); + this.instancesWithAvailableResources.put(queuedInstance.getResourceId(), queuedInstance); } } @@ -475,15 +478,18 @@ private Pair findInstance(Iterable requestedLocati return null; } - Iterator locations = requestedLocations == null ? null : requestedLocations.iterator(); + Iterator locations = requestedLocations == null ? null : requestedLocations.iterator(); if (locations != null && locations.hasNext()) { // we have a locality preference while (locations.hasNext()) { - Instance location = locations.next(); - if (location != null && this.instancesWithAvailableResources.remove(location)) { - return new ImmutablePair(location, Locality.LOCAL); + TaskManagerLocation location = locations.next(); + if (location != null) { + Instance instance = instancesWithAvailableResources.remove(location.getResourceID()); + if (instance != null) { + return new ImmutablePair(instance, Locality.LOCAL); + } } } @@ -492,14 +498,21 @@ private Pair findInstance(Iterable requestedLocati return null; } else { - Instance instanceToUse = this.instancesWithAvailableResources.poll(); - return new ImmutablePair(instanceToUse, Locality.NON_LOCAL); + // take the first instance from the instances with resources + Iterator instances = instancesWithAvailableResources.values().iterator(); + Instance instanceToUse = instances.next(); + instances.remove(); + + return new ImmutablePair<>(instanceToUse, Locality.NON_LOCAL); } } else { // no location preference, so use some instance - Instance instanceToUse = this.instancesWithAvailableResources.poll(); - return new ImmutablePair(instanceToUse, Locality.UNCONSTRAINED); + Iterator instances = instancesWithAvailableResources.values().iterator(); + Instance instanceToUse = instances.next(); + instances.remove(); + + return new ImmutablePair<>(instanceToUse, Locality.UNCONSTRAINED); } } @@ -570,7 +583,7 @@ private void handleNewSlot() { } } else { - this.instancesWithAvailableResources.add(instance); + this.instancesWithAvailableResources.put(instance.getResourceId(), instance); } } } @@ -643,11 +656,10 @@ public void newInstanceAvailable(Instance instance) { allInstancesByHost.put(instanceHostName, instanceSet); } instanceSet.add(instance); - - + // add it to the available resources and let potential waiters know - this.instancesWithAvailableResources.add(instance); - + this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + // add all slots as available for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) { newSlotAvailable(instance); @@ -681,8 +693,8 @@ private void removeInstance(Instance instance) { } allInstances.remove(instance); - instancesWithAvailableResources.remove(instance); - + instancesWithAvailableResources.remove(instance.getResourceId()); + String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet != null) { @@ -709,7 +721,7 @@ public int getNumberOfAvailableSlots() { synchronized (globalLock) { processNewlyAvailableInstances(); - for (Instance instance : instancesWithAvailableResources) { + for (Instance instance : instancesWithAvailableResources.values()) { count += instance.getNumberOfAvailableSlots(); } } @@ -781,9 +793,9 @@ private void processNewlyAvailableInstances() { synchronized (globalLock) { Instance instance; - while((instance = newlyAvailableInstances.poll()) != null){ - if(instance.hasResourcesAvailable()){ - instancesWithAvailableResources.add(instance); + while ((instance = newlyAvailableInstances.poll()) != null) { + if (instance.hasResourcesAvailable()) { + instancesWithAvailableResources.put(instance.getResourceId(), instance); } } } @@ -794,17 +806,17 @@ private void processNewlyAvailableInstances() { // Utilities // ------------------------------------------------------------------------ - private static String getHostnamesFromInstances(Iterable instances) { + private static String getHostnamesFromInstances(Iterable locations) { StringBuilder bld = new StringBuilder(); boolean successive = false; - for (Instance i : instances) { + for (TaskManagerLocation loc : locations) { if (successive) { bld.append(", "); } else { successive = true; } - bld.append(i.getInstanceConnectionInfo().getHostname()); + bld.append(loc.getHostname()); } return bld.toString(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java index 31bd3417f84c7..36e4072a620a6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java @@ -20,73 +20,125 @@ import org.apache.flink.runtime.instance.SimpleSlot; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * + */ public class SlotAllocationFuture { - + private final Object monitor = new Object(); - + private volatile SimpleSlot slot; - + private volatile SlotAllocationFutureAction action; - + // -------------------------------------------------------------------------------------------- + /** + * Creates a future that is uncompleted. + */ public SlotAllocationFuture() {} - + + /** + * Creates a future that is immediately completed. + * + * @param slot The task slot that completes the future. + */ public SlotAllocationFuture(SimpleSlot slot) { this.slot = slot; } - + // -------------------------------------------------------------------------------------------- - - public SimpleSlot waitTillAllocated() throws InterruptedException { - return waitTillAllocated(0); - } - - public SimpleSlot waitTillAllocated(long timeout) throws InterruptedException { + + public SimpleSlot waitTillCompleted() throws InterruptedException { synchronized (monitor) { while (slot == null) { - monitor.wait(timeout); + monitor.wait(); + } + return slot; + } + } + + public SimpleSlot waitTillCompleted(long timeout, TimeUnit timeUnit) throws InterruptedException, TimeoutException { + checkArgument(timeout >= 0, "timeout may not be negative"); + checkNotNull(timeUnit, "timeUnit"); + + if (timeout == 0) { + return waitTillCompleted(); + } else { + final long deadline = System.nanoTime() + timeUnit.toNanos(timeout); + long millisToWait; + + synchronized (monitor) { + while (slot == null && (millisToWait = (deadline - System.nanoTime()) / 1_000_000) > 0) { + monitor.wait(millisToWait); + } + + if (slot != null) { + return slot; + } else { + throw new TimeoutException(); + } } - + } + } + + /** + * Gets the slot from this future. This method throws an exception, if the future has not been completed. + * This method never blocks. + * + * @return The slot with which this future was completed. + * @throws IllegalStateException Thrown, if this method is called before the future is completed. + */ + public SimpleSlot get() { + final SimpleSlot slot = this.slot; + if (slot != null) { return slot; + } else { + throw new IllegalStateException("The future is not complete - not slot available"); } } - + public void setFutureAction(SlotAllocationFutureAction action) { + checkNotNull(action); + synchronized (monitor) { - if (this.action != null) { - throw new IllegalStateException("Future already has an action registered."); - } - + checkState(this.action == null, "Future already has an action registered."); + this.action = action; - + if (this.slot != null) { action.slotAllocated(this.slot); } } } - + + /** + * Completes the future with a slot. + */ public void setSlot(SimpleSlot slot) { - if (slot == null) { - throw new NullPointerException(); - } - + checkNotNull(slot); + synchronized (monitor) { - if (this.slot != null) { - throw new IllegalStateException("The future has already been assigned a slot."); - } - + checkState(this.slot == null, "The future has already been assigned a slot."); + this.slot = slot; monitor.notifyAll(); - + if (action != null) { action.slotAllocated(slot); } } } - + // -------------------------------------------------------------------------------------------- - + @Override public String toString() { return slot == null ? "PENDING" : "DONE"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java new file mode 100644 index 0000000000000..ad9c784c3407e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java @@ -0,0 +1,29 @@ +/* + * 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.jobmanager.slots; + +import org.apache.flink.runtime.instance.Slot; + +/** + * Interface for components that hold slots and to which slots get released / recycled. + */ +public interface SlotOwner { + + boolean returnAllocatedSlot(Slot slot); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index e39d7d285e486..d7f00771e1a64 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -107,7 +107,7 @@ public TaskManagerLocation(ResourceID resourceID, InetAddress inetAddress, int d } this.stringRepresentation = String.format( - "TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); + "%s @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); } // ------------------------------------------------------------------------ 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 b706a1ae1a0e0..d6f00249e4c39 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 @@ -650,7 +650,7 @@ class JobManager( val taskId = execution.getVertex.getParallelSubtaskIndex val host = if (slot != null) { - slot.getInstance().getInstanceConnectionInfo.getHostname + slot.getTaskManagerLocation().getHostname() } else { null } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala index 6a9b4904f35da..3947b1738260b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala @@ -249,7 +249,7 @@ trait TestingJobManagerLike extends FlinkActor { } else { sender ! decorateMessage( WorkingTaskManager( - Some(resource.getInstance().getActorGateway) + Some(resource.getTaskManagerActorGateway()) ) ) } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index cf7cf5800ec7c..d8bd6cbe9cad5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -117,7 +117,8 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti ActorGateway actorGateway = mock(ActorGateway.class); when(simpleSlot.isAlive()).thenReturn(true); - when(simpleSlot.getInstance()).thenReturn(instance); + when(simpleSlot.getTaskManagerID()).thenReturn(instance.getResourceId()); + when(simpleSlot.getTaskManagerLocation()).thenReturn(instance.getInstanceConnectionInfo()); when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); when(simpleSlot.getRoot()).thenReturn(rootSlot); @@ -152,6 +153,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti assertNotNull(metric); assertTrue(metric instanceof Gauge); + @SuppressWarnings("unchecked") Gauge restartingTime = (Gauge) metric; // check that the restarting time is 0 since it's the initial start diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 91472ae2fdbb5..a1f3345541b53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -92,8 +92,8 @@ public void testScheduleWithConstraint1() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3)); + vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -106,14 +106,14 @@ public void testScheduleWithConstraint1() { assertNotNull(slot1); assertNotNull(slot2); - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); assertNotNull(target1); assertNotNull(target2); - assertTrue(target1 == instance1 || target1 == instance2); - assertTrue(target2 == instance3); + assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); + assertEquals(target2, instance3.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -165,8 +165,8 @@ public void testScheduleWithConstraint2() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3)); - vertices[1].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -179,14 +179,11 @@ public void testScheduleWithConstraint2() { assertNotNull(slot1); assertNotNull(slot2); - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance3); - assertTrue(target2 == instance1 || target2 == instance2); + assertTrue(target1 == instance3.getResourceId()); + assertTrue(target2 == instance1.getResourceId() || target2 == instance2.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -242,8 +239,8 @@ public void testScheduleWithConstraintAndSlotSharing() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3)); + vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -255,15 +252,12 @@ public void testScheduleWithConstraintAndSlotSharing() { assertNotNull(slot1); assertNotNull(slot2); - - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); - - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance1 || target1 == instance2); - assertTrue(target2 == instance3); + + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); + + assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); + assertTrue(target2 == instance3.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -310,7 +304,7 @@ public void testScheduleWithUnfulfillableConstraint() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); try { @@ -380,7 +374,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); try { @@ -420,7 +414,7 @@ public void testArchivingClearsFields() { ExecutionVertex ev = eg.getAllVertices().get(vertex.getID()).getTaskVertices()[0]; Instance instance = ExecutionGraphTestUtils.getInstance(DummyActorGateway.INSTANCE); - ev.setLocationConstraintHosts(Collections.singletonList(instance)); + ev.setLocationConstraintHosts(Collections.singletonList(instance.getInstanceConnectionInfo())); assertNotNull(ev.getPreferredLocations()); assertEquals(instance, ev.getPreferredLocations().iterator().next()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 8709395edd9f7..2c40e89325dd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.junit.Test; @@ -38,7 +39,9 @@ * Tests for the allocation, properties, and release of shared slots. */ public class SharedSlotsTest { - + + private static final Iterable NO_LOCATION = Collections.emptySet(); + @Test public void allocateAndReleaseEmptySlot() { try { @@ -116,20 +119,20 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { SlotSharingGroupAssignment assignment = sharingGroup.getTaskAssignment(); Instance instance = SchedulerTestUtils.getRandomInstance(1); - + // allocate a shared slot SharedSlot sharedSlot = instance.allocateSharedSlot(jobId, assignment); - + // allocate a series of sub slots - + SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, vid1); assertNotNull(sub1); - + assertNull(sub1.getExecutedVertex()); assertEquals(Locality.LOCAL, sub1.getLocality()); assertEquals(1, sub1.getNumberLeaves()); assertEquals(vid1, sub1.getGroupID()); - assertEquals(instance, sub1.getInstance()); + assertEquals(instance.getResourceId(), sub1.getTaskManagerID()); assertEquals(jobId, sub1.getJobID()); assertEquals(sharedSlot, sub1.getParent()); assertEquals(sharedSlot, sub1.getRoot()); @@ -141,14 +144,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); assertNotNull(sub2); assertNull(sub2.getExecutedVertex()); assertEquals(Locality.UNCONSTRAINED, sub2.getLocality()); assertEquals(1, sub2.getNumberLeaves()); assertEquals(vid2, sub2.getGroupID()); - assertEquals(instance, sub2.getInstance()); + assertEquals(instance.getResourceId(), sub2.getTaskManagerID()); assertEquals(jobId, sub2.getJobID()); assertEquals(sharedSlot, sub2.getParent()); assertEquals(sharedSlot, sub2.getRoot()); @@ -160,14 +163,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance)); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getInstanceConnectionInfo())); assertNotNull(sub3); assertNull(sub3.getExecutedVertex()); assertEquals(Locality.LOCAL, sub3.getLocality()); assertEquals(1, sub3.getNumberLeaves()); assertEquals(vid3, sub3.getGroupID()); - assertEquals(instance, sub3.getInstance()); + assertEquals(instance.getResourceId(), sub3.getTaskManagerID()); assertEquals(jobId, sub3.getJobID()); assertEquals(sharedSlot, sub3.getParent()); assertEquals(sharedSlot, sub3.getRoot()); @@ -180,14 +183,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); SimpleSlot sub4 = assignment.getSlotForTask(vid4, - Collections.singleton(SchedulerTestUtils.getRandomInstance(1))); + Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getInstanceConnectionInfo())); assertNotNull(sub4); assertNull(sub4.getExecutedVertex()); assertEquals(Locality.NON_LOCAL, sub4.getLocality()); assertEquals(1, sub4.getNumberLeaves()); assertEquals(vid4, sub4.getGroupID()); - assertEquals(instance, sub4.getInstance()); + assertEquals(instance.getResourceId(), sub4.getTaskManagerID()); assertEquals(jobId, sub4.getJobID()); assertEquals(sharedSlot, sub4.getParent()); assertEquals(sharedSlot, sub4.getRoot()); @@ -229,7 +232,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { } @Test - public void allocateSimpleSlotsAndReleaseFromleaves() { + public void allocateSimpleSlotsAndReleaseFromLeaves() { try { JobID jobId = new JobID(); JobVertexID vid1 = new JobVertexID(); @@ -247,8 +250,8 @@ public void allocateSimpleSlotsAndReleaseFromleaves() { // allocate a series of sub slots SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION); assertNotNull(sub1); assertNotNull(sub2); @@ -332,7 +335,7 @@ public void allocateAndReleaseInMixedOrder() { // allocate a series of sub slots SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); assertNotNull(sub1); assertNotNull(sub2); @@ -353,7 +356,7 @@ public void allocateAndReleaseInMixedOrder() { assertEquals(1, assignment.getNumberOfSlots()); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.emptySet()); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION); assertNotNull(sub3); assertEquals(2, sharedSlot.getNumberLeaves()); @@ -431,7 +434,7 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(1, sharedSlot.getNumberLeaves()); // get the first slot in the nested shared slot from the co-location constraint - SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); assertEquals(2, sharedSlot.getNumberLeaves()); assertNotNull(constraint.getSharedSlot()); @@ -447,20 +450,20 @@ public void testAllocateAndReleaseTwoLevels() { assertFalse(constraint.isAssigned()); // re-allocate the head slot - headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); constraint.lockLocation(); assertNotNull(constraint.getSharedSlot()); assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance, constraint.getLocation()); + assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); - SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); assertEquals(constraint.getSharedSlot(), headSlot.getParent()); assertEquals(constraint.getSharedSlot(), tailSlot.getParent()); - SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); + SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); assertEquals(4, sharedSlot.getNumberLeaves()); // we release our co-location constraint tasks @@ -472,23 +475,23 @@ public void testAllocateAndReleaseTwoLevels() { assertTrue(tailSlot.isReleased()); assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance, constraint.getLocation()); + assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); // we should have resources again for the co-location constraint assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // re-allocate head and tail from the constraint - headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); - tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + headSlot = assignment.getSlotForTask(constraint, NO_LOCATION); + tailSlot = assignment.getSlotForTask(constraint, NO_LOCATION); assertEquals(4, sharedSlot.getNumberLeaves()); assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // verify some basic properties of the slots - assertEquals(instance, sourceSlot.getInstance()); - assertEquals(instance, headSlot.getInstance()); - assertEquals(instance, tailSlot.getInstance()); - assertEquals(instance, sinkSlot.getInstance()); + assertEquals(instance.getResourceId(), sourceSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), headSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), tailSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), sinkSlot.getTaskManagerID()); assertEquals(sourceId, sourceSlot.getGroupID()); assertEquals(sinkId, sinkSlot.getGroupID()); @@ -564,11 +567,11 @@ public void testReleaseTwoLevelsFromRoot() { // get the first simple slot SimpleSlot sourceSlot = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, sourceId); - SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot headSlot = assignment.getSlotForTask(constraint, NO_LOCATION); constraint.lockLocation(); - SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot tailSlot = assignment.getSlotForTask(constraint, NO_LOCATION); - SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); + SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, NO_LOCATION); assertEquals(4, sharedSlot.getNumberLeaves()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index 676b2a34e1c6f..5b7d18a3c20ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -27,9 +27,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -41,17 +43,17 @@ public void scheduleAllSharedAndCoLocated() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + scheduler.newInstanceAvailable(getRandomInstance(2)); scheduler.newInstanceAvailable(getRandomInstance(2)); scheduler.newInstanceAvailable(getRandomInstance(2)); - + assertEquals(6, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint c1 = new CoLocationConstraint(ccg); CoLocationConstraint c2 = new CoLocationConstraint(ccg); @@ -59,7 +61,7 @@ public void scheduleAllSharedAndCoLocated() { CoLocationConstraint c4 = new CoLocationConstraint(ccg); CoLocationConstraint c5 = new CoLocationConstraint(ccg); CoLocationConstraint c6 = new CoLocationConstraint(ccg); - + // schedule 4 tasks from the first vertex group SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1)); SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2)); @@ -86,7 +88,7 @@ public void scheduleAllSharedAndCoLocated() { assertNotNull(s10); assertNotNull(s11); assertNotNull(s12); - + // check that each slot got exactly two tasks assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); @@ -100,20 +102,20 @@ public void scheduleAllSharedAndCoLocated() { assertEquals(2, s10.getRoot().getNumberLeaves()); assertEquals(2, s11.getRoot().getNumberLeaves()); assertEquals(2, s12.getRoot().getNumberLeaves()); - - assertEquals(s1.getInstance(), s5.getInstance()); - assertEquals(s2.getInstance(), s6.getInstance()); - assertEquals(s3.getInstance(), s7.getInstance()); - assertEquals(s4.getInstance(), s10.getInstance()); - assertEquals(s8.getInstance(), s11.getInstance()); - assertEquals(s9.getInstance(), s12.getInstance()); - - assertEquals(c1.getLocation(), s1.getInstance()); - assertEquals(c2.getLocation(), s2.getInstance()); - assertEquals(c3.getLocation(), s3.getInstance()); - assertEquals(c4.getLocation(), s4.getInstance()); - assertEquals(c5.getLocation(), s8.getInstance()); - assertEquals(c6.getLocation(), s9.getInstance()); + + assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); + assertEquals(s3.getTaskManagerID(), s7.getTaskManagerID()); + assertEquals(s4.getTaskManagerID(), s10.getTaskManagerID()); + assertEquals(s8.getTaskManagerID(), s11.getTaskManagerID()); + assertEquals(s9.getTaskManagerID(), s12.getTaskManagerID()); + + assertEquals(c1.getLocation(), s1.getTaskManagerLocation()); + assertEquals(c2.getLocation(), s2.getTaskManagerLocation()); + assertEquals(c3.getLocation(), s3.getTaskManagerLocation()); + assertEquals(c4.getLocation(), s4.getTaskManagerLocation()); + assertEquals(c5.getLocation(), s8.getTaskManagerLocation()); + assertEquals(c6.getLocation(), s9.getTaskManagerLocation()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -186,14 +188,14 @@ public void scheduleWithIntermediateRelease() { SimpleSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1))); - Instance loc = s1.getInstance(); + ResourceID taskManager = s1.getTaskManagerID(); s1.releaseSlot(); s2.releaseSlot(); sSolo.releaseSlot(); SimpleSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1)); - assertEquals(loc, sNew.getInstance()); + assertEquals(taskManager, sNew.getTaskManagerID()); assertEquals(2, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -296,10 +298,10 @@ public void scheduleMixedCoLocationSlotSharing() { scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup)); scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup)); - assertEquals(s21.getInstance(), s34.getInstance()); - assertEquals(s22.getInstance(), s31.getInstance()); - assertEquals(s23.getInstance(), s32.getInstance()); - assertEquals(s24.getInstance(), s33.getInstance()); + assertEquals(s21.getTaskManagerID(), s34.getTaskManagerID()); + assertEquals(s22.getTaskManagerID(), s31.getTaskManagerID()); + assertEquals(s23.getTaskManagerID(), s32.getTaskManagerID()); + assertEquals(s24.getTaskManagerID(), s33.getTaskManagerID()); assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -320,10 +322,13 @@ public void testGetsNonLocalFromSharingGroupFirst() { JobVertexID jid3 = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -336,29 +341,29 @@ public void testGetsNonLocalFromSharingGroupFirst() { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); // schedule something into the shared group so that both instances are in the sharing group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); // schedule one locally to instance 1 - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc1)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1)); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // schedule something into the assigned co-location constraints and check that they override the // other preferences - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, i2), sharingGroup, cc1)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1)); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2)); // check that each slot got three assertEquals(3, s1.getRoot().getNumberLeaves()); assertEquals(3, s2.getRoot().getNumberLeaves()); - assertEquals(s1.getInstance(), s3.getInstance()); - assertEquals(s2.getInstance(), s4.getInstance()); - assertEquals(s1.getInstance(), s5.getInstance()); - assertEquals(s2.getInstance(), s6.getInstance()); + assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -389,10 +394,13 @@ public void testSlotReleasedInBetween() { JobVertexID jid2 = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -404,8 +412,8 @@ public void testSlotReleasedInBetween() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); s1.releaseSlot(); s2.releaseSlot(); @@ -413,12 +421,12 @@ public void testSlotReleasedInBetween() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // still preserves the previous instance mapping) - assertEquals(i1, s3.getInstance()); - assertEquals(i2, s4.getInstance()); + assertEquals(i1.getResourceId(), s3.getTaskManagerID()); + assertEquals(i2.getResourceId(), s4.getTaskManagerID()); s3.releaseSlot(); s4.releaseSlot(); @@ -443,10 +451,13 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { JobVertexID jidx = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -458,8 +469,8 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); s1.releaseSlot(); s2.releaseSlot(); @@ -471,7 +482,7 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { SimpleSlot sb = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2))); try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1)); + scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); fail("should not be able to find a resource"); } catch (NoResourceAvailableException e) { // good @@ -504,7 +515,9 @@ public void testScheduleOutOfOrder() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -519,18 +532,18 @@ public void testScheduleOutOfOrder() { // schedule something from the second job vertex id before the first is filled, // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2)); // check that each slot got three assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); - assertEquals(s1.getInstance(), s3.getInstance()); - assertEquals(s2.getInstance(), s4.getInstance()); + assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -566,7 +579,10 @@ public void nonColocationFollowsCoLocation() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -578,11 +594,11 @@ public void nonColocationFollowsCoLocation() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup)); // check that each slot got two assertEquals(2, s1.getRoot().getNumberLeaves()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 2ee53d93601c3..25498c4e0ef0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -18,26 +18,28 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; -import static org.junit.Assert.*; - +import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.testingUtils.TestingUtils; + import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.runtime.instance.Instance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the {@link Scheduler} when scheduling individual tasks. @@ -254,7 +256,7 @@ public void run() { List slotsAfter = new ArrayList(); for (SlotAllocationFuture future : allAllocatedSlots) { - slotsAfter.add(future.waitTillAllocated()); + slotsAfter.add(future.waitTillCompleted()); } assertEquals("All instances should have available slots.", NUM_INSTANCES, @@ -294,7 +296,7 @@ public void testScheduleWithDyingInstances() { i2.markDead(); for (SimpleSlot slot : slots) { - if (slot.getInstance() == i2) { + if (slot.getOwner() == i2) { assertTrue(slot.isCanceled()); } else { assertFalse(slot.isCanceled()); @@ -345,37 +347,37 @@ public void testSchedulingLocation() { scheduler.newInstanceAvailable(i3); // schedule something on an arbitrary instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.emptyList()))); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new Instance[0]))); // figure out how we use the location hints - Instance first = s1.getInstance(); + Instance first = (Instance) s1.getOwner(); Instance second = first != i1 ? i1 : i2; Instance third = first == i3 ? i2 : i3; // something that needs to go to the first instance again - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.singletonList(s1.getInstance())))); - assertEquals(first, s2.getInstance()); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation()))); + assertEquals(first, s2.getOwner()); // first or second --> second, because first is full - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, second)))); - assertEquals(second, s3.getInstance()); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, second))); + assertEquals(second, s3.getOwner()); // first or third --> third (because first is full) - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(third, s4.getInstance()); - assertEquals(third, s5.getInstance()); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(third, s4.getOwner()); + assertEquals(third, s5.getOwner()); // first or third --> second, because all others are full - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(second, s6.getInstance()); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(second, s6.getOwner()); // release something on the first and second instance s2.releaseSlot(); s6.releaseSlot(); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(first, s7.getInstance()); + SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(first, s7.getOwner()); assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments()); assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index 3e90123b4e7c8..a683834553d32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -18,12 +18,13 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; -import static org.apache.flink.runtime.testutils.CommonTestUtils.sleepUninterruptibly; -import static org.junit.Assert.*; +import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.testingUtils.TestingUtils; + +import org.junit.Test; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -31,11 +32,16 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.junit.Test; -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.jobgraph.JobVertexID; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; +import static org.apache.flink.runtime.testutils.CommonTestUtils.sleepUninterruptibly; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the scheduler when scheduling tasks in slot sharing groups. @@ -102,10 +108,10 @@ public void scheduleSingleVertexType() { // make sure we have two slots on the first instance, and two on the second int c = 0; - c += (s5.getInstance() == i1) ? 1 : -1; - c += (s6.getInstance() == i1) ? 1 : -1; - c += (s7.getInstance() == i1) ? 1 : -1; - c += (s8.getInstance() == i1) ? 1 : -1; + c += (s5.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s6.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s7.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s8.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; assertEquals(0, c); // release all @@ -625,20 +631,23 @@ public void testLocalizedAssignment1() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -647,8 +656,8 @@ public void testLocalizedAssignment1() { assertEquals(1, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); assertNotNull(s3); assertNotNull(s4); @@ -675,20 +684,23 @@ public void testLocalizedAssignment2() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -697,8 +709,8 @@ public void testLocalizedAssignment2() { assertEquals(2, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); assertNotNull(s3); assertNotNull(s4); @@ -725,25 +737,27 @@ public void testLocalizedAssignment3() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule until the one instance is full - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, i1), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup)); // schedule two more with preference of same instance --> need to go to other instance - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, i1), sharingGroup)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, i1), sharingGroup)); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup)); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -757,12 +771,12 @@ public void testLocalizedAssignment3() { assertEquals(0, i1.getNumberOfAvailableSlots()); assertEquals(0, i2.getNumberOfAvailableSlots()); - assertEquals(i1, s1.getInstance()); - assertEquals(i1, s2.getInstance()); - assertEquals(i1, s3.getInstance()); - assertEquals(i1, s4.getInstance()); - assertEquals(i2, s5.getInstance()); - assertEquals(i2, s6.getInstance()); + assertEquals(i1.getResourceId(), s1.getTaskManagerID()); + assertEquals(i1.getResourceId(), s2.getTaskManagerID()); + assertEquals(i1.getResourceId(), s3.getTaskManagerID()); + assertEquals(i1.getResourceId(), s4.getTaskManagerID()); + assertEquals(i2.getResourceId(), s5.getTaskManagerID()); + assertEquals(i2.getResourceId(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index 1736c390bdb99..b7c46657f3262 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -23,9 +23,11 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -82,8 +84,21 @@ public static Execution getDummyTask() { return execution; } + + public static Execution getTestVertex(Instance... preferredInstances) { + List locations = new ArrayList<>(preferredInstances.length); + for (Instance i : preferredInstances) { + locations.add(i.getInstanceConnectionInfo()); + } + return getTestVertex(locations); + } + + public static Execution getTestVertex(TaskManagerLocation... preferredLocations) { + return getTestVertex(Arrays.asList(preferredLocations)); + } + - public static Execution getTestVertex(Iterable preferredLocations) { + public static Execution getTestVertex(Iterable preferredLocations) { ExecutionVertex vertex = mock(ExecutionVertex.class); when(vertex.getPreferredLocations()).thenReturn(preferredLocations); @@ -112,20 +127,22 @@ public static Execution getTestVertex(JobVertexID jid, int taskIndex, int numTas return execution; } - - public static Execution getTestVertexWithLocation(JobVertexID jid, int taskIndex, int numTasks, Instance... locations) { + + public static Execution getTestVertexWithLocation( + JobVertexID jid, int taskIndex, int numTasks, TaskManagerLocation... locations) { + ExecutionVertex vertex = mock(ExecutionVertex.class); - + when(vertex.getPreferredLocations()).thenReturn(Arrays.asList(locations)); when(vertex.getJobId()).thenReturn(new JobID()); when(vertex.getJobvertexId()).thenReturn(jid); when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex); when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks); when(vertex.toString()).thenReturn("TEST-VERTEX"); - + Execution execution = mock(Execution.class); when(execution.getVertex()).thenReturn(vertex); - + return execution; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java index d678531da4602..d9c100c080ed3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java @@ -18,14 +18,19 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.junit.Assert.*; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.SimpleSlot; + +import org.junit.Test; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.api.common.JobID; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; public class SlotAllocationFutureTest { @@ -46,9 +51,14 @@ public void slotAllocated(SimpleSlot slot) {} } catch (IllegalStateException e) { // expected } - - final SimpleSlot slot1 = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); - final SimpleSlot slot2 = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance1 = SchedulerTestUtils.getRandomInstance(1); + final Instance instance2 = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot slot1 = new SimpleSlot(new JobID(), instance1, + instance1.getInstanceConnectionInfo(), 0, instance1.getActorGateway(), null, null); + final SimpleSlot slot2 = new SimpleSlot(new JobID(), instance2, + instance2.getInstanceConnectionInfo(), 0, instance2.getActorGateway(), null, null); future.setSlot(slot1); try { @@ -71,7 +81,11 @@ public void setWithAction() { // action before the slot { final AtomicInteger invocations = new AtomicInteger(); - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); @@ -91,7 +105,10 @@ public void slotAllocated(SimpleSlot slot) { // slot before action { final AtomicInteger invocations = new AtomicInteger(); - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); @@ -120,8 +137,11 @@ public void setSync() { { final AtomicInteger invocations = new AtomicInteger(); final AtomicBoolean error = new AtomicBoolean(); - - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); @@ -130,7 +150,7 @@ public void setSync() { @Override public void run() { try { - SimpleSlot syncSlot = future.waitTillAllocated(); + SimpleSlot syncSlot = future.waitTillCompleted(); if (syncSlot == null || syncSlot != thisSlot) { error.set(true); return; @@ -158,12 +178,15 @@ public void run() { // setting slot before syncing { - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); - SimpleSlot retrieved = future.waitTillAllocated(); + SimpleSlot retrieved = future.waitTillCompleted(); assertNotNull(retrieved); assertEquals(thisSlot, retrieved); diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 3566ab83f2ac1..4590b48ea334c 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -18,21 +18,17 @@ package org.apache.flink.yarn -import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.TaskManagerInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManagerConfiguration, TaskManager} -import org.apache.flink.runtime.util.ProcessShutDownThread -import scala.concurrent.duration._ - -/** An extension of the TaskManager that listens for additional YARN related - * messages. - */ +/** + * An extension of the TaskManager that listens for additional YARN related + * messages. + */ class YarnTaskManager( config: TaskManagerConfiguration, resourceID: ResourceID, From 7b30d8f6246af60b7fceebb0cdde761aee89a714 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 31 Aug 2016 13:59:01 +0200 Subject: [PATCH 3/4] [FLINK-4490] [distributed coordination] (part 3) Rename methods on 'Instance' to have more intuitive names getResourceID() --> getTaskManagerID() getInstanceConnectionInfo() --> getTaskManagerLocation() --- .../handlers/TaskManagersHandler.java | 2 +- .../flink/runtime/instance/Instance.java | 32 +- .../runtime/instance/InstanceManager.java | 2 +- .../jobmanager/scheduler/Scheduler.java | 18 +- .../flink/runtime/jobmanager/JobManager.scala | 2 +- .../ExecutionGraphMetricsTest.java | 404 +++++++++--------- .../runtime/instance/InstanceManagerTest.java | 2 +- .../runtime/instance/SharedSlotsTest.java | 24 +- .../SpilledSubpartitionViewTest.java | 4 +- .../scheduler/CoLocationConstraintTest.java | 6 +- .../ScheduleWithCoLocationHintTest.java | 22 +- .../scheduler/SchedulerSlotSharingTest.java | 30 +- .../scheduler/SchedulerTestUtils.java | 2 +- .../scheduler/SlotAllocationFutureTest.java | 12 +- 14 files changed, 288 insertions(+), 274 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java index b60cd105b1fe3..b5e90888bf146 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java @@ -85,7 +85,7 @@ public String handleRequest(Map pathParams, Map gen.writeStartObject(); gen.writeStringField("id", instance.getId().toString()); gen.writeStringField("path", instance.getActorGateway().path()); - gen.writeNumberField("dataPort", instance.getInstanceConnectionInfo().dataPort()); + gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort()); gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat()); gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots()); gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index fe46895c4500b..a7f0c21edc8f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -52,13 +52,13 @@ public class Instance implements SlotOwner { private final ActorGateway actorGateway; /** The instance connection information for the data transfer. */ - private final TaskManagerLocation connectionInfo; + private final TaskManagerLocation location; /** A description of the resources of the task manager */ private final HardwareDescription resources; - /** The ID identifies the resource the task manager runs on */ - private final ResourceID resourceId; + /** The ID identifies the task manager */ + private final ResourceID taskManagerId; /** The ID identifying the taskManager. */ private final InstanceID instanceId; @@ -90,22 +90,22 @@ public class Instance implements SlotOwner { * Constructs an instance reflecting a registered TaskManager. * * @param actorGateway The actor gateway to communicate with the remote instance - * @param connectionInfo The remote connection where the task manager receives requests. - * @param resourceId The resource id which denotes the resource the task manager uses. + * @param location The remote connection where the task manager receives requests. + * @param taskManagerId The resource id which denotes the resource the task manager uses. * @param id The id under which the taskManager is registered. * @param resources The resources available on the machine. * @param numberOfSlots The number of task slots offered by this taskManager. */ public Instance( ActorGateway actorGateway, - TaskManagerLocation connectionInfo, - ResourceID resourceId, + TaskManagerLocation location, + ResourceID taskManagerId, InstanceID id, HardwareDescription resources, int numberOfSlots) { this.actorGateway = actorGateway; - this.connectionInfo = connectionInfo; - this.resourceId = resourceId; + this.location = location; + this.taskManagerId = taskManagerId; this.instanceId = id; this.resources = resources; this.numberOfSlots = numberOfSlots; @@ -120,8 +120,8 @@ public Instance( // Properties // -------------------------------------------------------------------------------------------- - public ResourceID getResourceId() { - return resourceId; + public ResourceID getTaskManagerID() { + return taskManagerId; } public InstanceID getId() { @@ -246,7 +246,7 @@ public SimpleSlot allocateSimpleSlot(JobID jobID) throws InstanceDiedException { return null; } else { - SimpleSlot slot = new SimpleSlot(jobID, this, connectionInfo, nextSlot, actorGateway); + SimpleSlot slot = new SimpleSlot(jobID, this, location, nextSlot, actorGateway); allocatedSlots.add(slot); return slot; } @@ -284,7 +284,7 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha } else { SharedSlot slot = new SharedSlot( - jobID, this, connectionInfo, nextSlot, actorGateway, sharingGroupAssignment); + jobID, this, location, nextSlot, actorGateway, sharingGroupAssignment); allocatedSlots.add(slot); return slot; } @@ -355,8 +355,8 @@ public ActorGateway getActorGateway() { return actorGateway; } - public TaskManagerLocation getInstanceConnectionInfo() { - return connectionInfo; + public TaskManagerLocation getTaskManagerLocation() { + return location; } public int getNumberOfAvailableSlots() { @@ -405,7 +405,7 @@ public void removeSlotListener() { @Override public String toString() { - return String.format("%s @ %s - %d slots - URL: %s", instanceId, connectionInfo.getHostname(), + return String.format("%s @ %s - %d slots - URL: %s", instanceId, location.getHostname(), numberOfSlots, (actorGateway != null ? actorGateway.path() : "No instance gateway")); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index e7a45376d4631..aaa786b2b5643 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -217,7 +217,7 @@ public void unregisterTaskManager(ActorRef instanceID, boolean terminated){ registeredHostsByConnection.remove(host); registeredHostsById.remove(instance.getId()); - registeredHostsByResource.remove(instance.getResourceId()); + registeredHostsByResource.remove(instance.getTaskManagerID()); if (terminated) { deadHosts.add(instance.getActorGateway().actor()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index b481b55ec12b4..734972dd8dc72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -363,7 +363,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getTaskManagerID(), instanceToUse); } if (slot != null) { @@ -425,7 +425,7 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getTaskManagerID(), instanceToUse); } if (sharedSlot != null) { @@ -469,7 +469,7 @@ private Pair findInstance(Iterable requ while (this.newlyAvailableInstances.size() > 0) { Instance queuedInstance = this.newlyAvailableInstances.poll(); if (queuedInstance != null) { - this.instancesWithAvailableResources.put(queuedInstance.getResourceId(), queuedInstance); + this.instancesWithAvailableResources.put(queuedInstance.getTaskManagerID(), queuedInstance); } } @@ -583,7 +583,7 @@ private void handleNewSlot() { } } else { - this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + this.instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); } } } @@ -649,7 +649,7 @@ public void newInstanceAvailable(Instance instance) { instance.setSlotAvailabilityListener(this); // store the instance in the by-host-lookup - String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); + String instanceHostName = instance.getTaskManagerLocation().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet == null) { instanceSet = new HashSet(); @@ -658,7 +658,7 @@ public void newInstanceAvailable(Instance instance) { instanceSet.add(instance); // add it to the available resources and let potential waiters know - this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + this.instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); // add all slots as available for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) { @@ -693,9 +693,9 @@ private void removeInstance(Instance instance) { } allInstances.remove(instance); - instancesWithAvailableResources.remove(instance.getResourceId()); + instancesWithAvailableResources.remove(instance.getTaskManagerID()); - String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); + String instanceHostName = instance.getTaskManagerLocation().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet != null) { instanceSet.remove(instance); @@ -795,7 +795,7 @@ private void processNewlyAvailableInstances() { while ((instance = newlyAvailableInstances.poll()) != null) { if (instance.hasResourcesAvailable()) { - instancesWithAvailableResources.put(instance.getResourceId(), instance); + instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); } } } 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 d6f00249e4c39..dfcb83ac280ec 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 @@ -349,7 +349,7 @@ class JobManager( currentResourceManager = Option(msg.resourceManager()) val taskManagerResources = instanceManager.getAllRegisteredInstances.asScala.map( - instance => instance.getResourceId).toList.asJava + instance => instance.getTaskManagerID).toList.asJava // confirm registration and send known task managers with their resource ids sender ! decorateMessage(new RegisterResourceManagerSuccessful(self, taskManagerResources)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index d8bd6cbe9cad5..d5520fddaf590 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; @@ -47,8 +48,10 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.Matchers; + import scala.concurrent.ExecutionContext$; import scala.concurrent.Future$; import scala.concurrent.duration.FiniteDuration; @@ -60,7 +63,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; @@ -76,200 +80,210 @@ public class ExecutionGraphMetricsTest extends TestLogger { */ @Test public void testExecutionGraphRestartTimeMetric() throws JobException, IOException, InterruptedException { - // setup execution graph with mocked scheduling logic - int parallelism = 1; - - JobVertex jobVertex = new JobVertex("TestVertex"); - jobVertex.setParallelism(parallelism); - jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jobGraph = new JobGraph("Test Job", jobVertex); - - Configuration config = new Configuration(); - config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test"); - config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName()); - - Configuration jobConfig = new Configuration(); - - FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); - - MetricRegistry metricRegistry = new MetricRegistry(config); - - assertTrue(metricRegistry.getReporters().size() == 1); - - MetricReporter reporter = metricRegistry.getReporters().get(0); - - assertTrue(reporter instanceof TestingReporter); - - TestingReporter testingReporter = (TestingReporter) reporter; - - MetricGroup metricGroup = new JobManagerMetricGroup(metricRegistry, "localhost"); - - Scheduler scheduler = mock(Scheduler.class); - - SimpleSlot simpleSlot = mock(SimpleSlot.class); - - Instance instance = mock(Instance.class); - - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - - Slot rootSlot = mock(Slot.class); - - ActorGateway actorGateway = mock(ActorGateway.class); - - when(simpleSlot.isAlive()).thenReturn(true); - when(simpleSlot.getTaskManagerID()).thenReturn(instance.getResourceId()); - when(simpleSlot.getTaskManagerLocation()).thenReturn(instance.getInstanceConnectionInfo()); - when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); - when(simpleSlot.getRoot()).thenReturn(rootSlot); - - when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); - - when(instance.getInstanceConnectionInfo()).thenReturn(taskManagerLocation); - when(instance.getActorGateway()).thenReturn(actorGateway); - when(taskManagerLocation.getHostname()).thenReturn("localhost"); - - when(rootSlot.getSlotNumber()).thenReturn(0); - - when(actorGateway.ask(Matchers.any(Object.class), Matchers.any(FiniteDuration.class))).thenReturn(Future$.MODULE$.successful(Messages.getAcknowledge())); - - TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); - - ExecutionGraph executionGraph = new ExecutionGraph( - ExecutionContext$.MODULE$.fromExecutor(new ForkJoinPool()), - jobGraph.getJobID(), - jobGraph.getName(), - jobConfig, - new SerializedValue(null), - timeout, - testingRestartStrategy, - Collections.emptyList(), - Collections.emptyList(), - getClass().getClassLoader(), - metricGroup); - - // get restarting time metric - Metric metric = testingReporter.getMetric(ExecutionGraph.RESTARTING_TIME_METRIC_NAME); - - assertNotNull(metric); - assertTrue(metric instanceof Gauge); - - @SuppressWarnings("unchecked") - Gauge restartingTime = (Gauge) metric; - - // check that the restarting time is 0 since it's the initial start - assertTrue(0L == restartingTime.getValue()); - - executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); - - // start execution - executionGraph.scheduleForExecution(scheduler); - - assertTrue(0L == restartingTime.getValue()); - - List executionIDs = new ArrayList<>(); - - for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { - executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); - } - - // tell execution graph that the tasks are in state running --> job status switches to state running - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); - } - - assertEquals(JobStatus.RUNNING, executionGraph.getState()); - - assertTrue(0L == restartingTime.getValue()); - - // fail the job so that it goes into state restarting - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); - } - - assertEquals(JobStatus.RESTARTING, executionGraph.getState()); - - long firstRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); - - // wait some time so that the restarting time gauge shows a value different from 0 - Thread.sleep(50); - - long previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time is monotonically increasing - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime >= previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - // check that we have measured some restarting time - assertTrue(previousRestartingTime > 0); - - // restart job - testingRestartStrategy.restartExecutionGraph(); - - executionIDs.clear(); - - for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { - executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); - } - - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); - } - - assertEquals(JobStatus.RUNNING, executionGraph.getState()); - - assertTrue(firstRestartingTimestamp != 0); - - previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time does not increase after we've reached the running state - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime == previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - // fail job again - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); - } - - assertEquals(JobStatus.RESTARTING, executionGraph.getState()); - - long secondRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); - - assertTrue(firstRestartingTimestamp != secondRestartingTimestamp); - - Thread.sleep(50); - - previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time is increasing again - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime >= previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - assertTrue(previousRestartingTime > 0); - - // now lets fail the job while it is in restarting and see whether the restarting time then stops to increase - executionGraph.fail(new Exception()); - - assertEquals(JobStatus.FAILED, executionGraph.getState()); - - previousRestartingTime = restartingTime.getValue(); - - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime == previousRestartingTime); - previousRestartingTime = currentRestartingTime; + final ExecutorService executor = Executors.newCachedThreadPool(); + try { + // setup execution graph with mocked scheduling logic + int parallelism = 1; + + JobVertex jobVertex = new JobVertex("TestVertex"); + jobVertex.setParallelism(parallelism); + jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); + JobGraph jobGraph = new JobGraph("Test Job", jobVertex); + + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName()); + + Configuration jobConfig = new Configuration(); + + FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); + + MetricRegistry metricRegistry = new MetricRegistry(config); + + assertTrue(metricRegistry.getReporters().size() == 1); + + MetricReporter reporter = metricRegistry.getReporters().get(0); + + assertTrue(reporter instanceof TestingReporter); + + TestingReporter testingReporter = (TestingReporter) reporter; + + MetricGroup metricGroup = new JobManagerMetricGroup(metricRegistry, "localhost"); + + Scheduler scheduler = mock(Scheduler.class); + + ResourceID taskManagerId = ResourceID.generate(); + + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(taskManagerId); + when(taskManagerLocation.getHostname()).thenReturn("localhost"); + + ActorGateway actorGateway = mock(ActorGateway.class); + + Instance instance = mock(Instance.class); + when(instance.getTaskManagerLocation()).thenReturn(taskManagerLocation); + when(instance.getTaskManagerID()).thenReturn(taskManagerId); + when(instance.getActorGateway()).thenReturn(actorGateway); + + Slot rootSlot = mock(Slot.class); + + SimpleSlot simpleSlot = mock(SimpleSlot.class); + when(simpleSlot.isAlive()).thenReturn(true); + when(simpleSlot.getTaskManagerLocation()).thenReturn(taskManagerLocation); + when(simpleSlot.getTaskManagerID()).thenReturn(taskManagerId); + when(simpleSlot.getTaskManagerActorGateway()).thenReturn(actorGateway); + when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); + when(simpleSlot.getRoot()).thenReturn(rootSlot); + + when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); + + + + when(rootSlot.getSlotNumber()).thenReturn(0); + + when(actorGateway.ask(Matchers.any(Object.class), Matchers.any(FiniteDuration.class))).thenReturn(Future$.MODULE$.successful(Messages.getAcknowledge())); + + TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); + + ExecutionGraph executionGraph = new ExecutionGraph( + ExecutionContext$.MODULE$.fromExecutor(executor), + jobGraph.getJobID(), + jobGraph.getName(), + jobConfig, + new SerializedValue(null), + timeout, + testingRestartStrategy, + Collections.emptyList(), + Collections.emptyList(), + getClass().getClassLoader(), + metricGroup); + + // get restarting time metric + Metric metric = testingReporter.getMetric(ExecutionGraph.RESTARTING_TIME_METRIC_NAME); + + assertNotNull(metric); + assertTrue(metric instanceof Gauge); + + @SuppressWarnings("unchecked") + Gauge restartingTime = (Gauge) metric; + + // check that the restarting time is 0 since it's the initial start + assertTrue(0L == restartingTime.getValue()); + + executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); + + // start execution + executionGraph.scheduleForExecution(scheduler); + + assertTrue(0L == restartingTime.getValue()); + + List executionIDs = new ArrayList<>(); + + for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { + executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + // tell execution graph that the tasks are in state running --> job status switches to state running + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); + } + + assertEquals(JobStatus.RUNNING, executionGraph.getState()); + + assertTrue(0L == restartingTime.getValue()); + + // fail the job so that it goes into state restarting + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); + } + + assertEquals(JobStatus.RESTARTING, executionGraph.getState()); + + long firstRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); + + // wait some time so that the restarting time gauge shows a value different from 0 + Thread.sleep(50); + + long previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time is monotonically increasing + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime >= previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + // check that we have measured some restarting time + assertTrue(previousRestartingTime > 0); + + // restart job + testingRestartStrategy.restartExecutionGraph(); + + executionIDs.clear(); + + for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { + executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); + } + + assertEquals(JobStatus.RUNNING, executionGraph.getState()); + + assertTrue(firstRestartingTimestamp != 0); + + previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time does not increase after we've reached the running state + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime == previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + // fail job again + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); + } + + assertEquals(JobStatus.RESTARTING, executionGraph.getState()); + + long secondRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); + + assertTrue(firstRestartingTimestamp != secondRestartingTimestamp); + + Thread.sleep(50); + + previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time is increasing again + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime >= previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + assertTrue(previousRestartingTime > 0); + + // now lets fail the job while it is in restarting and see whether the restarting time then stops to increase + executionGraph.fail(new Exception()); + + assertEquals(JobStatus.FAILED, executionGraph.getState()); + + previousRestartingTime = restartingTime.getValue(); + + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime == previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + } finally { + executor.shutdownNow(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java index f1ed9601e8ec3..42d26afd987e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java @@ -102,7 +102,7 @@ public void testInstanceRegistering() { HashSet(); for(Instance instance: instances){ - taskManagerLocations.add(instance.getInstanceConnectionInfo()); + taskManagerLocations.add(instance.getTaskManagerLocation()); } assertTrue(taskManagerLocations.contains(ici1)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 2c40e89325dd4..0edef5e2dc051 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -132,7 +132,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(Locality.LOCAL, sub1.getLocality()); assertEquals(1, sub1.getNumberLeaves()); assertEquals(vid1, sub1.getGroupID()); - assertEquals(instance.getResourceId(), sub1.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub1.getTaskManagerID()); assertEquals(jobId, sub1.getJobID()); assertEquals(sharedSlot, sub1.getParent()); assertEquals(sharedSlot, sub1.getRoot()); @@ -151,7 +151,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(Locality.UNCONSTRAINED, sub2.getLocality()); assertEquals(1, sub2.getNumberLeaves()); assertEquals(vid2, sub2.getGroupID()); - assertEquals(instance.getResourceId(), sub2.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub2.getTaskManagerID()); assertEquals(jobId, sub2.getJobID()); assertEquals(sharedSlot, sub2.getParent()); assertEquals(sharedSlot, sub2.getRoot()); @@ -163,14 +163,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getInstanceConnectionInfo())); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getTaskManagerLocation())); assertNotNull(sub3); assertNull(sub3.getExecutedVertex()); assertEquals(Locality.LOCAL, sub3.getLocality()); assertEquals(1, sub3.getNumberLeaves()); assertEquals(vid3, sub3.getGroupID()); - assertEquals(instance.getResourceId(), sub3.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub3.getTaskManagerID()); assertEquals(jobId, sub3.getJobID()); assertEquals(sharedSlot, sub3.getParent()); assertEquals(sharedSlot, sub3.getRoot()); @@ -183,14 +183,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); SimpleSlot sub4 = assignment.getSlotForTask(vid4, - Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getInstanceConnectionInfo())); + Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getTaskManagerLocation())); assertNotNull(sub4); assertNull(sub4.getExecutedVertex()); assertEquals(Locality.NON_LOCAL, sub4.getLocality()); assertEquals(1, sub4.getNumberLeaves()); assertEquals(vid4, sub4.getGroupID()); - assertEquals(instance.getResourceId(), sub4.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub4.getTaskManagerID()); assertEquals(jobId, sub4.getJobID()); assertEquals(sharedSlot, sub4.getParent()); assertEquals(sharedSlot, sub4.getRoot()); @@ -456,7 +456,7 @@ public void testAllocateAndReleaseTwoLevels() { assertNotNull(constraint.getSharedSlot()); assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); + assertEquals(instance.getTaskManagerLocation(), constraint.getLocation()); SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); @@ -475,7 +475,7 @@ public void testAllocateAndReleaseTwoLevels() { assertTrue(tailSlot.isReleased()); assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); + assertEquals(instance.getTaskManagerLocation(), constraint.getLocation()); // we should have resources again for the co-location constraint assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); @@ -488,10 +488,10 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // verify some basic properties of the slots - assertEquals(instance.getResourceId(), sourceSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), headSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), tailSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), sinkSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sourceSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), headSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), tailSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sinkSlot.getTaskManagerID()); assertEquals(sourceId, sourceSlot.getGroupID()); assertEquals(sinkId, sinkSlot.getGroupID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java index fff7bc69a6f70..5722cac11a6f0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition; -import com.google.common.collect.Lists; import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; @@ -36,6 +35,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -121,7 +121,7 @@ public void testReadMultipleFilesWithSingleBufferPool() throws Exception { } } - final List> results = Lists.newArrayList(); + final List> results = new ArrayList<>(); // Submit the consuming tasks for (ResultSubpartitionView view : readers) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java index 3bd4368ec79c3..1344aef289054 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java @@ -140,7 +140,7 @@ public void testAssignSlotAndLockLocation() { // now, the location is assigned and we have a location assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); // release the slot slot2_1.releaseSlot(); @@ -148,7 +148,7 @@ public void testAssignSlotAndLockLocation() { // we should still have a location assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); // we can not assign a different location try { @@ -167,7 +167,7 @@ public void testAssignSlotAndLockLocation() { assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index 5b7d18a3c20ea..eab4fea69a372 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -326,8 +326,8 @@ public void testGetsNonLocalFromSharingGroupFirst() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -398,8 +398,8 @@ public void testSlotReleasedInBetween() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -425,8 +425,8 @@ public void testSlotReleasedInBetween() { SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // still preserves the previous instance mapping) - assertEquals(i1.getResourceId(), s3.getTaskManagerID()); - assertEquals(i2.getResourceId(), s4.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s4.getTaskManagerID()); s3.releaseSlot(); s4.releaseSlot(); @@ -455,8 +455,8 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -516,7 +516,7 @@ public void testScheduleOutOfOrder() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -580,8 +580,8 @@ public void nonColocationFollowsCoLocation() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index a683834553d32..fd0523b985932 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -108,10 +108,10 @@ public void scheduleSingleVertexType() { // make sure we have two slots on the first instance, and two on the second int c = 0; - c += (s5.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s6.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s7.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s8.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s5.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s6.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s7.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s8.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; assertEquals(0, c); // release all @@ -637,8 +637,8 @@ public void testLocalizedAssignment1() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -690,8 +690,8 @@ public void testLocalizedAssignment2() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -743,7 +743,7 @@ public void testLocalizedAssignment3() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -771,12 +771,12 @@ public void testLocalizedAssignment3() { assertEquals(0, i1.getNumberOfAvailableSlots()); assertEquals(0, i2.getNumberOfAvailableSlots()); - assertEquals(i1.getResourceId(), s1.getTaskManagerID()); - assertEquals(i1.getResourceId(), s2.getTaskManagerID()); - assertEquals(i1.getResourceId(), s3.getTaskManagerID()); - assertEquals(i1.getResourceId(), s4.getTaskManagerID()); - assertEquals(i2.getResourceId(), s5.getTaskManagerID()); - assertEquals(i2.getResourceId(), s6.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s1.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s2.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index b7c46657f3262..2c743cae404b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -88,7 +88,7 @@ public static Execution getDummyTask() { public static Execution getTestVertex(Instance... preferredInstances) { List locations = new ArrayList<>(preferredInstances.length); for (Instance i : preferredInstances) { - locations.add(i.getInstanceConnectionInfo()); + locations.add(i.getTaskManagerLocation()); } return getTestVertex(locations); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java index d9c100c080ed3..ea0d2cc25e243 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java @@ -56,9 +56,9 @@ public void slotAllocated(SimpleSlot slot) {} final Instance instance2 = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot slot1 = new SimpleSlot(new JobID(), instance1, - instance1.getInstanceConnectionInfo(), 0, instance1.getActorGateway(), null, null); + instance1.getTaskManagerLocation(), 0, instance1.getActorGateway(), null, null); final SimpleSlot slot2 = new SimpleSlot(new JobID(), instance2, - instance2.getInstanceConnectionInfo(), 0, instance2.getActorGateway(), null, null); + instance2.getTaskManagerLocation(), 0, instance2.getActorGateway(), null, null); future.setSlot(slot1); try { @@ -85,7 +85,7 @@ public void setWithAction() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); @@ -108,7 +108,7 @@ public void slotAllocated(SimpleSlot slot) { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); @@ -141,7 +141,7 @@ public void setSync() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); @@ -181,7 +181,7 @@ public void run() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); From e7efae594015e2788f840b0fc3c685880fd35e85 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 31 Aug 2016 13:52:45 +0200 Subject: [PATCH 4/4] [FLINK-4525] [core] (followup) Remove remaining redundant code for pre-defined strictly local assignments. --- .../executiongraph/ExecutionVertex.java | 13 - .../VertexLocationConstraintTest.java | 456 ------------------ 2 files changed, 469 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.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 8da3dc11a001d..d2837a7dd9ccd 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 @@ -95,8 +95,6 @@ public class ExecutionVertex { private volatile Execution currentExecution; // this field must never be null - private volatile List locationConstraintInstances; - private volatile boolean scheduleLocalOnly; // -------------------------------------------------------------------------------------------- @@ -345,10 +343,6 @@ else if (numSources < parallelism) { } } - public void setLocationConstraintHosts(List instances) { - this.locationConstraintInstances = instances; - } - public void setScheduleLocalOnly(boolean scheduleLocalOnly) { if (scheduleLocalOnly && inputEdges != null && inputEdges.length > 0) { throw new IllegalArgumentException("Strictly local scheduling is only supported for sources."); @@ -370,12 +364,6 @@ public boolean isScheduleLocalOnly() { * @return The preferred locations for this vertex execution, or null, if there is no preference. */ public Iterable getPreferredLocations() { - // if we have hard location constraints, use those - List constraintInstances = this.locationConstraintInstances; - if (constraintInstances != null && !constraintInstances.isEmpty()) { - return constraintInstances; - } - // otherwise, base the preferred locations on the input connections if (inputEdges == null) { return Collections.emptySet(); @@ -564,7 +552,6 @@ public void prepareForArchiving() throws IllegalStateException { this.resultPartitions = null; this.inputEdges = null; this.locationConstraint = null; - this.locationConstraintInstances = null; } public void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo){ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java deleted file mode 100644 index a1f3345541b53..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ /dev/null @@ -1,456 +0,0 @@ -/* - * 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 static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -import java.net.InetAddress; -import java.util.Arrays; -import java.util.Collections; -import java.util.concurrent.TimeUnit; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.instance.DummyActorGateway; -import org.apache.flink.runtime.instance.HardwareDescription; -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; -import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.util.SerializedValue; -import org.junit.Test; - -import scala.concurrent.duration.FiniteDuration; - -public class VertexLocationConstraintTest { - - private static final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS); - - @Test - public void testScheduleWithConstraint1() { - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); - assertEquals(target2, instance3.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithConstraint2() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertTrue(target1 == instance3.getResourceId()); - assertTrue(target2 == instance1.getResourceId() || target2 == instance2.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithConstraintAndSlotSharing() { - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex1 = new JobVertex("v1", new JobVertexID()); - JobVertex jobVertex2 = new JobVertex("v2", new JobVertexID()); - jobVertex1.setInvokableClass(DummyInvokable.class); - jobVertex2.setInvokableClass(DummyInvokable.class); - jobVertex1.setParallelism(2); - jobVertex2.setParallelism(3); - - SlotSharingGroup sharingGroup = new SlotSharingGroup(); - jobVertex1.setSlotSharingGroup(sharingGroup); - jobVertex2.setSlotSharingGroup(sharingGroup); - - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); - assertTrue(target2 == instance3.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithUnfulfillableConstraint() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(1); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); - vertices[0].setScheduleLocalOnly(true); - - try { - ejv.scheduleAll(scheduler, false); - fail("This should fail with a NoResourceAvailableException"); - } - catch (NoResourceAvailableException e) { - // bam! we are good... - assertTrue(e.getMessage().contains(hostname2)); - } - catch (Exception e) { - fail("Wrong exception type"); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithUnfulfillableConstraintInSharingGroup() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - - // prepare the execution graph - JobVertex jobVertex1 = new JobVertex("v1", new JobVertexID()); - JobVertex jobVertex2 = new JobVertex("v2", new JobVertexID()); - - jobVertex1.setInvokableClass(DummyInvokable.class); - jobVertex2.setInvokableClass(DummyInvokable.class); - - jobVertex1.setParallelism(1); - jobVertex2.setParallelism(1); - - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); - - SlotSharingGroup sharingGroup = new SlotSharingGroup(); - jobVertex1.setSlotSharingGroup(sharingGroup); - jobVertex2.setSlotSharingGroup(sharingGroup); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); - vertices[0].setScheduleLocalOnly(true); - - try { - ejv.scheduleAll(scheduler, false); - fail("This should fail with a NoResourceAvailableException"); - } - catch (NoResourceAvailableException e) { - // bam! we are good... - assertTrue(e.getMessage().contains(hostname2)); - } - catch (Exception e) { - fail("Wrong exception type"); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testArchivingClearsFields() { - try { - JobVertex vertex = new JobVertex("test vertex", new JobVertexID()); - JobGraph jg = new JobGraph("test job", vertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(vertex)); - - ExecutionVertex ev = eg.getAllVertices().get(vertex.getID()).getTaskVertices()[0]; - - Instance instance = ExecutionGraphTestUtils.getInstance(DummyActorGateway.INSTANCE); - ev.setLocationConstraintHosts(Collections.singletonList(instance.getInstanceConnectionInfo())); - - assertNotNull(ev.getPreferredLocations()); - assertEquals(instance, ev.getPreferredLocations().iterator().next()); - - // transition to a final state - eg.fail(new Exception()); - - eg.prepareForArchiving(); - - assertTrue(ev.getPreferredLocations() == null || !ev.getPreferredLocations().iterator().hasNext()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - public static Instance getInstance(byte[] ipAddress, int dataPort, String hostname) throws Exception { - HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); - - TaskManagerLocation connection = mock(TaskManagerLocation.class); - when(connection.address()).thenReturn(InetAddress.getByAddress(ipAddress)); - when(connection.dataPort()).thenReturn(dataPort); - when(connection.addressString()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); - when(connection.getHostname()).thenReturn(hostname); - when(connection.getFQDNHostname()).thenReturn(hostname); - - return new Instance( - new ExecutionGraphTestUtils.SimpleActorGateway( - TestingUtils.defaultExecutionContext()), - connection, - ResourceID.generate(), - new InstanceID(), - hardwareDescription, - 1); - } -}