You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2016/09/02 15:36:05 UTC

[1/8] flink git commit: [FLINK-4525] [core] (followup) Remove remaining redundant code for pre-defined strictly local assignments.

Repository: flink
Updated Branches:
  refs/heads/master c251efca2 -> fb8f2c935


[FLINK-4525] [core] (followup) Remove remaining redundant code for pre-defined strictly local assignments.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/fb8f2c93
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fb8f2c93
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fb8f2c93

Branch: refs/heads/master
Commit: fb8f2c935e96c9300a7584e310eb96e8a1f32f7f
Parents: eac6088
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Aug 31 13:52:45 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionVertex.java |  13 -
 .../VertexLocationConstraintTest.java           | 456 -------------------
 2 files changed, 469 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fb8f2c93/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index f02647e..88e1b88 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
@@ -97,8 +97,6 @@ public class ExecutionVertex {
 
 	private volatile Execution currentExecution;	// this field must never be null
 
-	private volatile List<TaskManagerLocation> locationConstraintInstances;
-
 	private volatile boolean scheduleLocalOnly;
 
 	// --------------------------------------------------------------------------------------------
@@ -351,10 +349,6 @@ public class ExecutionVertex {
 		}
 	}
 
-	public void setLocationConstraintHosts(List<TaskManagerLocation> 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.");
@@ -376,12 +370,6 @@ public class ExecutionVertex {
 	 * @return The preferred locations for this vertex execution, or null, if there is no preference.
 	 */
 	public Iterable<TaskManagerLocation> getPreferredLocations() {
-		// if we have hard location constraints, use those
-		List<TaskManagerLocation> 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();
@@ -570,7 +558,6 @@ public class ExecutionVertex {
 		this.resultPartitions = null;
 		this.inputEdges = null;
 		this.locationConstraint = null;
-		this.locationConstraintInstances = null;
 	}
 
 	public void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo){

http://git-wip-us.apache.org/repos/asf/flink/blob/fb8f2c93/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java
----------------------------------------------------------------------
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 a1f3345..0000000
--- 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);
-	}
-}


[2/8] flink git commit: [FLINK-4490] [distributed coordination] (part 2) Make slots independent of 'Instance'.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java
----------------------------------------------------------------------
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 8709395..2c40e89 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.CoLocationGroup;
 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 @@ import static org.junit.Assert.*;
  * Tests for the allocation, properties, and release of shared slots.
  */
 public class SharedSlotsTest {
-	
+
+	private static final Iterable<TaskManagerLocation> NO_LOCATION = Collections.emptySet();
+
 	@Test
 	public void allocateAndReleaseEmptySlot() {
 		try {
@@ -116,20 +119,20 @@ public class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3));
 			assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4));
 			
-			SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.<Instance>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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 	}
 
 	@Test
-	public void allocateSimpleSlotsAndReleaseFromleaves() {
+	public void allocateSimpleSlotsAndReleaseFromLeaves() {
 		try {
 			JobID jobId = new JobID();
 			JobVertexID vid1 = new JobVertexID();
@@ -247,8 +250,8 @@ public class SharedSlotsTest {
 			// allocate a series of sub slots
 
 			SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1);
-			SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.<Instance>emptySet());
-			SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.<Instance>emptySet());
+			SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION);
+			SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION);
 			
 			assertNotNull(sub1);
 			assertNotNull(sub2);
@@ -332,7 +335,7 @@ public class SharedSlotsTest {
 			// allocate a series of sub slots
 
 			SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1);
-			SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.<Instance>emptySet());
+			SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION);
 
 			assertNotNull(sub1);
 			assertNotNull(sub2);
@@ -353,7 +356,7 @@ public class SharedSlotsTest {
 			assertEquals(1, assignment.getNumberOfSlots());
 			
 			
-			SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.<Instance>emptySet());
+			SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION);
 			assertNotNull(sub3);
 			
 			assertEquals(2, sharedSlot.getNumberLeaves());
@@ -431,7 +434,7 @@ public class SharedSlotsTest {
 			assertEquals(1, sharedSlot.getNumberLeaves());
 			
 			// get the first slot in the nested shared slot from the co-location constraint
-			SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.<Instance>emptySet());
+			SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.<TaskManagerLocation>emptySet());
 			assertEquals(2, sharedSlot.getNumberLeaves());
 
 			assertNotNull(constraint.getSharedSlot());
@@ -447,20 +450,20 @@ public class SharedSlotsTest {
 			assertFalse(constraint.isAssigned());
 			
 			// re-allocate the head slot
-			headSlot = assignment.getSlotForTask(constraint, Collections.<Instance>emptySet());
+			headSlot = assignment.getSlotForTask(constraint, Collections.<TaskManagerLocation>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.<Instance>emptySet());
+			SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.<TaskManagerLocation>emptySet());
 			
 			assertEquals(constraint.getSharedSlot(), headSlot.getParent());
 			assertEquals(constraint.getSharedSlot(), tailSlot.getParent());
 			
-			SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.<Instance>emptySet());
+			SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.<TaskManagerLocation>emptySet());
 			assertEquals(4, sharedSlot.getNumberLeaves());
 			
 			// we release our co-location constraint tasks
@@ -472,23 +475,23 @@ public class SharedSlotsTest {
 			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.<Instance>emptySet());
-			tailSlot = assignment.getSlotForTask(constraint, Collections.<Instance>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 class SharedSlotsTest {
 			// get the first simple slot
 			SimpleSlot sourceSlot = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, sourceId);
 			
-			SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.<Instance>emptySet());
+			SimpleSlot headSlot = assignment.getSlotForTask(constraint, NO_LOCATION);
 			constraint.lockLocation();
-			SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.<Instance>emptySet());
+			SimpleSlot tailSlot = assignment.getSlotForTask(constraint, NO_LOCATION);
 			
-			SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.<Instance>emptySet());
+			SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, NO_LOCATION);
 			
 			assertEquals(4, sharedSlot.getNumberLeaves());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
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 676b2a3..5b7d18a 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.assertNotNull;
 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 class ScheduleWithCoLocationHintTest {
 		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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			
 			Instance i1 = getRandomInstance(1);
 			Instance i2 = getRandomInstance(1);
-			
+
+			TaskManagerLocation loc1 = i1.getInstanceConnectionInfo();
+
 			scheduler.newInstanceAvailable(i2);
 			scheduler.newInstanceAvailable(i1);
 			
@@ -519,18 +532,18 @@ public class ScheduleWithCoLocationHintTest {
 			// 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 class ScheduleWithCoLocationHintTest {
 			
 			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 class ScheduleWithCoLocationHintTest {
 			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());

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
----------------------------------------------------------------------
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 2ee53d9..25498c4 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 class SchedulerIsolatedTasksTest {
 
 			List<SimpleSlot> slotsAfter = new ArrayList<SimpleSlot>();
 			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 class SchedulerIsolatedTasksTest {
 			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 class SchedulerIsolatedTasksTest {
 			scheduler.newInstanceAvailable(i3);
 			
 			// schedule something on an arbitrary instance
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.<Instance>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());

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
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 3e90123..a683834 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.Executors;
 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 class SchedulerSlotSharingTest {
 			
 			// 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 class SchedulerSlotSharingTest {
 		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 class SchedulerSlotSharingTest {
 			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 class SchedulerSlotSharingTest {
 		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 class SchedulerSlotSharingTest {
 			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 class SchedulerSlotSharingTest {
 		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 class SchedulerSlotSharingTest {
 			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());

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
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 99360e2..eef27a8 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 static org.mockito.Mockito.when;
 
 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 class SchedulerTestUtils {
 		
 		return execution;
 	}
+
+	public static Execution getTestVertex(Instance... preferredInstances) {
+		List<TaskManagerLocation> 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<Instance> preferredLocations) {
+	public static Execution getTestVertex(Iterable<TaskManagerLocation> preferredLocations) {
 		ExecutionVertex vertex = mock(ExecutionVertex.class);
 		
 		when(vertex.getPreferredLocations()).thenReturn(preferredLocations);
@@ -113,10 +128,12 @@ public class SchedulerTestUtils {
 		
 		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);
@@ -124,10 +141,10 @@ public class SchedulerTestUtils {
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
 		when(vertex.getMaxParallelism()).thenReturn(numTasks);
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
-		
+
 		Execution execution = mock(Execution.class);
 		when(execution.getVertex()).thenReturn(vertex);
-		
+
 		return execution;
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
----------------------------------------------------------------------
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 d678531..d9c100c 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 class SlotAllocationFutureTest {
 			} 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 class SlotAllocationFutureTest {
 			// 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 class SlotAllocationFutureTest {
 			// 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 class SlotAllocationFutureTest {
 			{
 				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 class SlotAllocationFutureTest {
 					@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 class SlotAllocationFutureTest {
 			
 			// 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);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
----------------------------------------------------------------------
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 3307568..0c2ca1a 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,20 +24,24 @@ import org.apache.flink.configuration.Configuration;
 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.HardwareDescription;
 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.BeforeClass;
 import org.junit.Test;
-import org.mockito.Mockito;
+
 import scala.Option;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * It cases which test the interaction of the resource manager with job manager and task managers.
@@ -79,12 +83,13 @@ public class ResourceManagerITCase extends TestLogger {
 
 			ResourceID resourceID = ResourceID.generate();
 
+			TaskManagerLocation location = mock(TaskManagerLocation.class);
+			when(location.getResourceID()).thenReturn(resourceID);
+
+			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
+
 			jobManager.tell(
-				new RegistrationMessages.RegisterTaskManager(
-					resourceID,
-					Mockito.mock(TaskManagerLocation.class),
-					null,
-					1),
+				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
 				me);
 
 			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);


[7/8] flink git commit: [FLINK-4490] [distributed coordination] (part 1) Change InstanceConnectionInfo to TaskManagerLocation

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java
new file mode 100644
index 0000000..9452b20
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.taskmanager;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.net.InetAddress;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the TaskManagerLocation, which identifies the location and connection
+ * information of a TaskManager.
+ */
+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");
+			when(address1.getHostName()).thenReturn("localhost");
+			when(address1.getHostAddress()).thenReturn("127.0.0.1");
+			when(address1.getAddress()).thenReturn(new byte[] {127, 0, 0, 1} );
+
+			InetAddress address2 = mock(InetAddress.class);
+			when(address2.getCanonicalHostName()).thenReturn("testhost1");
+			when(address2.getHostName()).thenReturn("testhost1");
+			when(address2.getHostAddress()).thenReturn("0.0.0.0");
+			when(address2.getAddress()).thenReturn(new byte[] {0, 0, 0, 0} );
+
+			InetAddress address3 = mock(InetAddress.class);
+			when(address3.getCanonicalHostName()).thenReturn("testhost2");
+			when(address3.getHostName()).thenReturn("testhost2");
+			when(address3.getHostAddress()).thenReturn("192.168.0.1");
+			when(address3.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, 0, 1} );
+
+			// one == four != two != three
+			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));
+			assertTrue(!one.equals(three));
+			assertTrue(!two.equals(three));
+			assertTrue(!three.equals(four));
+			
+			assertTrue(one.compareTo(four) == 0);
+			assertTrue(four.compareTo(one) == 0);
+			assertTrue(one.compareTo(two) != 0);
+			assertTrue(one.compareTo(three) != 0);
+			assertTrue(two.compareTo(three) != 0);
+			assertTrue(three.compareTo(four) != 0);
+			
+			{
+				int val = one.compareTo(two);
+				assertTrue(two.compareTo(one) == -val);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialization() {
+		try {
+			// without resolved hostname
+			{
+				TaskManagerLocation original = new TaskManagerLocation(
+						ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888);
+
+				TaskManagerLocation serCopy = InstantiationUtil.clone(original);
+				assertEquals(original, serCopy);
+			}
+						
+			// with resolved hostname
+			{
+				TaskManagerLocation original = new TaskManagerLocation(
+						ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871);
+				original.getFQDNHostname();
+
+				TaskManagerLocation serCopy = InstantiationUtil.clone(original);
+				assertEquals(original, serCopy);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testGetFQDNHostname() {
+		try {
+			TaskManagerLocation info1 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871);
+			assertNotNull(info1.getFQDNHostname());
+			
+			TaskManagerLocation info2 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888);
+			assertNotNull(info2.getFQDNHostname());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testGetHostname0() {
+		try {
+			InetAddress address = mock(InetAddress.class);
+			when(address.getCanonicalHostName()).thenReturn("worker2.cluster.mycompany.com");
+			when(address.getHostName()).thenReturn("worker2.cluster.mycompany.com");
+			when(address.getHostAddress()).thenReturn("127.0.0.1");
+
+			final TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871);
+			Assert.assertEquals("worker2", info.getHostname());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testGetHostname1() {
+		try {
+			InetAddress address = mock(InetAddress.class);
+			when(address.getCanonicalHostName()).thenReturn("worker10");
+			when(address.getHostName()).thenReturn("worker10");
+			when(address.getHostAddress()).thenReturn("127.0.0.1");
+
+			TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871);
+			Assert.assertEquals("worker10", info.getHostname());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testGetHostname2() {
+		try {
+			final String addressString = "192.168.254.254";
+
+			// we mock the addresses to save the times of the reverse name lookups
+			InetAddress address = mock(InetAddress.class);
+			when(address.getCanonicalHostName()).thenReturn("192.168.254.254");
+			when(address.getHostName()).thenReturn("192.168.254.254");
+			when(address.getHostAddress()).thenReturn("192.168.254.254");
+			when(address.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, (byte) 254, (byte) 254} );
+
+			TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 54152);
+
+			assertNotNull(info.getFQDNHostname());
+			assertTrue(info.getFQDNHostname().equals(addressString));
+
+			assertNotNull(info.getHostname());
+			assertTrue(info.getHostname().equals(addressString));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala
----------------------------------------------------------------------
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 7feb949..f9c9b63 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) {

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala
----------------------------------------------------------------------
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 73ab7eb..f9f294b 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 = {

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
----------------------------------------------------------------------
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 107801d..0c6264e 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
@@ -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.{TaskManager, TaskManagerConfiguration}
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation}
 
 /** An extension of the TaskManager that listens for additional YARN related
   * messages.
@@ -32,7 +31,7 @@ import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfigurati
 class YarnTaskManager(
     config: TaskManagerConfiguration,
     resourceID: ResourceID,
-    connectionInfo: InstanceConnectionInfo,
+    taskManagerLocation: TaskManagerLocation,
     memoryManager: MemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
@@ -41,7 +40,7 @@ class YarnTaskManager(
   extends TaskManager(
     config,
     resourceID,
-    connectionInfo,
+    taskManagerLocation,
     memoryManager,
     ioManager,
     network,


[3/8] flink git commit: [FLINK-4490] [distributed coordination] (part 2) Make slots independent of 'Instance'.

Posted by se...@apache.org.
[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.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/aaa474ad
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/aaa474ad
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/aaa474ad

Branch: refs/heads/master
Commit: aaa474ad8f1d638c3988697dd57446802142119b
Parents: 34cda87
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Aug 30 20:34:20 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../InputChannelDeploymentDescriptor.java       |  14 +-
 .../flink/runtime/executiongraph/Execution.java |  51 +++----
 .../runtime/executiongraph/ExecutionVertex.java |  17 ++-
 .../runtime/instance/HardwareDescription.java   |  28 ++--
 .../apache/flink/runtime/instance/Instance.java |  22 +--
 .../flink/runtime/instance/SharedSlot.java      |  45 ++++--
 .../flink/runtime/instance/SimpleSlot.java      |  35 +++--
 .../org/apache/flink/runtime/instance/Slot.java | 103 ++++++++++---
 .../instance/SlotSharingGroupAssignment.java    | 132 ++++++++---------
 .../scheduler/CoLocationConstraint.java         |  48 ++++---
 .../runtime/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 +++----
 .../flink/runtime/instance/SharedSlotsTest.java |  67 ++++-----
 .../ScheduleWithCoLocationHintTest.java         | 144 ++++++++++---------
 .../scheduler/SchedulerIsolatedTasksTest.java   |  52 +++----
 .../scheduler/SchedulerSlotSharingTest.java     | 102 +++++++------
 .../scheduler/SchedulerTestUtils.java           |  29 +++-
 .../scheduler/SlotAllocationFutureTest.java     |  51 +++++--
 .../resourcemanager/ResourceManagerITCase.java  |  17 ++-
 25 files changed, 741 insertions(+), 497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java
----------------------------------------------------------------------
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 f31febb..0912055 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;
 
@@ -88,6 +90,7 @@ public class InputChannelDeploymentDescriptor implements Serializable {
 	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
@@ -105,16 +108,17 @@ public class InputChannelDeploymentDescriptor implements Serializable {
 					(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);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
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 197999c..846df49 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.api.common.accumulators.Accumulator;
 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;
@@ -49,7 +49,6 @@ import org.apache.flink.runtime.messages.TaskMessages.TaskOperationResult;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.ExceptionUtils;
 
 import org.slf4j.Logger;
@@ -371,7 +370,7 @@ public class Execution {
 				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) {
@@ -381,7 +380,7 @@ public class Execution {
 
 			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(
@@ -393,9 +392,8 @@ public class Execution {
 
 			// 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<Object> deployAction = gateway.ask(new SubmitTask(deployment), timeout);
 
@@ -408,7 +406,7 @@ public class Execution {
 							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 {
@@ -437,7 +435,7 @@ public class Execution {
 		final SimpleSlot slot = this.assignedResource;
 
 		if (slot != null) {
-			final ActorGateway gateway = slot.getInstance().getActorGateway();
+			final ActorGateway gateway = slot.getTaskManagerActorGateway();
 
 			Future<Object> stopResult = gateway.retry(
 				new StopTask(attemptId),
@@ -590,24 +588,25 @@ public class Execution {
 						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);
@@ -916,7 +915,7 @@ public class Execution {
 
 		if (slot != null) {
 
-			final ActorGateway gateway = slot.getInstance().getActorGateway();
+			final ActorGateway gateway = slot.getTaskManagerActorGateway();
 
 			Future<Object> cancelResult = gateway.retry(
 				new CancelTask(attemptId),
@@ -946,14 +945,10 @@ public class Execution {
 		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));
 		}
 	}
 
@@ -968,15 +963,15 @@ public class Execution {
 			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<Object> 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);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index e5a115a..f02647e 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.PartialInputChannelDeploymentDescript
 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;
@@ -98,7 +97,7 @@ public class ExecutionVertex {
 
 	private volatile Execution currentExecution;	// this field must never be null
 
-	private volatile List<Instance> locationConstraintInstances;
+	private volatile List<TaskManagerLocation> locationConstraintInstances;
 
 	private volatile boolean scheduleLocalOnly;
 
@@ -352,7 +351,7 @@ public class ExecutionVertex {
 		}
 	}
 
-	public void setLocationConstraintHosts(List<Instance> instances) {
+	public void setLocationConstraintHosts(List<TaskManagerLocation> instances) {
 		this.locationConstraintInstances = instances;
 	}
 
@@ -376,9 +375,9 @@ public class ExecutionVertex {
 	 *
 	 * @return The preferred locations for this vertex execution, or null, if there is no preference.
 	 */
-	public Iterable<Instance> getPreferredLocations() {
+	public Iterable<TaskManagerLocation> getPreferredLocations() {
 		// if we have hard location constraints, use those
-		List<Instance> constraintInstances = this.locationConstraintInstances;
+		List<TaskManagerLocation> constraintInstances = this.locationConstraintInstances;
 		if (constraintInstances != null && !constraintInstances.isEmpty()) {
 			return constraintInstances;
 		}
@@ -388,8 +387,8 @@ public class ExecutionVertex {
 			return Collections.emptySet();
 		}
 		else {
-			Set<Instance> locations = new HashSet<Instance>();
-			Set<Instance> inputLocations = new HashSet<Instance>();
+			Set<TaskManagerLocation> locations = new HashSet<>();
+			Set<TaskManagerLocation> inputLocations = new HashSet<>();
 
 			// go over all inputs
 			for (int i = 0; i < inputEdges.length; i++) {
@@ -402,7 +401,7 @@ public class ExecutionVertex {
 						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();
@@ -495,7 +494,7 @@ public class ExecutionVertex {
 
 			// 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);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java
----------------------------------------------------------------------
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 bfcc1e5..9c1c5b7 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 final class HardwareDescription implements Serializable {
 	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 final class HardwareDescription implements Serializable {
 	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);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
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 598b32b..fe46895 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 java.util.Set;
 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 class Instance {
 				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 class Instance {
 				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 class Instance {
 	 * @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);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java
----------------------------------------------------------------------
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 ef62910..7f05604 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 @@ import java.util.Set;
  * <p><b>IMPORTANT:</b> 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.</p>
+ * 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 class SharedSlot extends Slot {
 	 * 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<Slot>();
 	}
 
@@ -148,7 +167,9 @@ public class SharedSlot extends Slot {
 	 */
 	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 @@ public class SharedSlot extends Slot {
 	 */
 	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;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java
----------------------------------------------------------------------
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 dbe961a..615138f 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 @@ package org.apache.flink.runtime.instance;
 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.
  *
  * <p>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.</p>
+ * 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 class SimpleSlot extends Slot {
 	 * 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 class SimpleSlot extends Slot {
 
 	@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 class SimpleSlot extends Slot {
 			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

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
----------------------------------------------------------------------
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 341ef95..451a9ec 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 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
  * <p>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.</p>
+ * 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 abstract class Slot {
 	}
 
 	/**
-	 * 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.
+	 *
+	 * <p>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 abstract class Slot {
 	 * 
 	 * @return The ID identifying the logical group of slots.
 	 */
+	@Nullable
 	public AbstractID getGroupID() {
 		return groupID;
 	}
@@ -158,10 +209,18 @@ public abstract class Slot {
 	 * 
 	 * @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 @@ public abstract class Slot {
 
 	@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) {

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
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 7d666fe..346cc77 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.List;
 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<SharedSlot> allSlots = new LinkedHashSet<SharedSlot>();
 
-	/** The slots available per vertex type (jid), keyed by instance, to make them locatable */
-	private final Map<AbstractID, Map<Instance, List<SharedSlot>>> availableSlotsPerJid = 
-			new LinkedHashMap<AbstractID, Map<Instance, List<SharedSlot>>>();
+	/** The slots available per vertex type (JobVertexId), keyed by TaskManager, to make them locatable */
+	private final Map<AbstractID, Map<ResourceID, List<SharedSlot>>> availableSlotsPerJid = new LinkedHashMap<>();
 
 
 	// --------------------------------------------------------------------------------------------
@@ -122,7 +123,7 @@ public class SlotSharingGroupAssignment {
 	 */
 	public int getNumberOfAvailableSlotsForGroup(AbstractID groupId) {
 		synchronized (lock) {
-			Map<Instance, List<SharedSlot>> available = availableSlotsPerJid.get(groupId);
+			Map<ResourceID, List<SharedSlot>> available = availableSlotsPerJid.get(groupId);
 
 			if (available != null) {
 				Set<SharedSlot> set = new HashSet<SharedSlot>();
@@ -148,37 +149,25 @@ public class SlotSharingGroupAssignment {
 	//  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 @@ public class SlotSharingGroupAssignment {
 				// can place a task into this slot.
 				boolean entryForNewJidExists = false;
 				
-				for (Map.Entry<AbstractID, Map<Instance, List<SharedSlot>>> entry : availableSlotsPerJid.entrySet()) {
+				for (Map.Entry<AbstractID, Map<ResourceID, List<SharedSlot>>> entry : availableSlotsPerJid.entrySet()) {
 					// there is already an entry for this groupID
 					if (entry.getKey().equals(groupIdForMap)) {
 						entryForNewJidExists = true;
 						continue;
 					}
 
-					Map<Instance, List<SharedSlot>> available = entry.getValue();
+					Map<ResourceID, List<SharedSlot>> 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<Instance, List<SharedSlot>>());
+					availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap<ResourceID, List<SharedSlot>>());
 				}
 
 				return subSlot;
@@ -287,18 +276,15 @@ public class SlotSharingGroupAssignment {
 
 	/**
 	 * 
-	 * @param vertexID
-	 * @param locationPreferences
-	 * @return
 	 */
-	SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable<Instance> locationPreferences) {
+	SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable<TaskManagerLocation> locationPreferences) {
 		synchronized (lock) {
-			Pair<SharedSlot, Locality> p = getSlotForTaskInternal(vertexID, locationPreferences, false);
+			Tuple2<SharedSlot, Locality> 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 class SlotSharingGroupAssignment {
 		return getSlotForTask(constraint, vertex.getPreferredLocations());
 	}
 	
-	SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable<Instance> locationPreferences) {
+	SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable<TaskManagerLocation> 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 @@ public class SlotSharingGroupAssignment {
 				if (previous == null) {
 					throw new IllegalStateException("Bug: Found assigned co-location constraint without a slot.");
 				}
-				
-				Instance location = previous.getInstance();
-				Pair<SharedSlot, Locality> p = getSlotForTaskInternal(constraint.getGroupId(),
-																		Collections.singleton(location), true);
+
+				TaskManagerLocation location = previous.getTaskManagerLocation();
+				Tuple2<SharedSlot, Locality> 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 @@ public class SlotSharingGroupAssignment {
 				// 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<SharedSlot, Locality> p =
+				Tuple2<SharedSlot, Locality> 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 @@ public class SlotSharingGroupAssignment {
 	}
 
 
-	private Pair<SharedSlot, Locality> getSlotForTaskInternal(AbstractID groupId,
-																Iterable<Instance> preferredLocations,
-																boolean localOnly)
+	private Tuple2<SharedSlot, Locality> getSlotForTaskInternal(
+			AbstractID groupId, Iterable<TaskManagerLocation> preferredLocations, boolean localOnly)
 	{
 		// check if there is anything at all in this group assignment
 		if (allSlots.isEmpty()) {
@@ -415,15 +401,15 @@ public class SlotSharingGroupAssignment {
 		}
 
 		// get the available slots for the group
-		Map<Instance, List<SharedSlot>> slotsForGroup = availableSlotsPerJid.get(groupId);
+		Map<ResourceID, List<SharedSlot>> slotsForGroup = availableSlotsPerJid.get(groupId);
 		
 		if (slotsForGroup == null) {
 			// we have a new group, so all slots are available
-			slotsForGroup = new LinkedHashMap<Instance, List<SharedSlot>>();
+			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 @@ public class SlotSharingGroupAssignment {
 		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<SharedSlot, Locality>(slot, Locality.LOCAL);
+					return new Tuple2<>(slot, Locality.LOCAL);
 				}
 			}
 		}
@@ -459,7 +445,7 @@ public class SlotSharingGroupAssignment {
 		SharedSlot slot;
 		while ((slot = pollFromMultiMap(slotsForGroup)) != null) {
 			if (slot.isAlive()) {
-				return new ImmutablePair<SharedSlot, Locality>(slot, locality);
+				return new Tuple2<>(slot, locality);
 			}
 		}
 		
@@ -510,7 +496,7 @@ public class SlotSharingGroupAssignment {
 							// for that group again. otherwise, the slot is part of a
 							// co-location group and nothing becomes immediately available
 
-							Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(groupID);
+							Map<ResourceID, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(groupID);
 
 							// sanity check
 							if (slotsForJid == null) {
@@ -518,7 +504,7 @@ public class SlotSharingGroupAssignment {
 										" 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 @@ public class SlotSharingGroupAssignment {
 	/**
 	 * 
 	 * <p><b>NOTE: This method must be called from within a scope that holds the lock.</b></p>
-	 * 
-	 * @param sharedSlot
 	 */
 	private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) {
 		// sanity check
@@ -576,7 +560,7 @@ public class SlotSharingGroupAssignment {
 		
 		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 @@ public class SlotSharingGroupAssignment {
 				
 				if (parentRemaining > 0) {
 					// the parent becomes available for the group again
-					Map<Instance, List<SharedSlot>> slotsForGroup = availableSlotsPerJid.get(groupID);
+					Map<ResourceID, List<SharedSlot>> slotsForGroup = availableSlotsPerJid.get(groupID);
 
 					// sanity check
 					if (slotsForGroup == null) {
@@ -600,7 +584,7 @@ public class SlotSharingGroupAssignment {
 								" when available slots indicated that all slots were available.");
 					}
 
-					putIntoMultiMap(slotsForGroup, parent.getInstance(), parent);
+					putIntoMultiMap(slotsForGroup, parent.getTaskManagerID(), parent);
 					
 				}
 				else {
@@ -620,7 +604,7 @@ public class SlotSharingGroupAssignment {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	private static void putIntoMultiMap(Map<Instance, List<SharedSlot>> map, Instance location, SharedSlot slot) {
+	private static void putIntoMultiMap(Map<ResourceID, List<SharedSlot>> map, ResourceID location, SharedSlot slot) {
 		List<SharedSlot> slotsForInstance = map.get(location);
 		if (slotsForInstance == null) {
 			slotsForInstance = new ArrayList<SharedSlot>();
@@ -629,7 +613,7 @@ public class SlotSharingGroupAssignment {
 		slotsForInstance.add(slot);
 	}
 	
-	private static SharedSlot removeFromMultiMap(Map<Instance, List<SharedSlot>> map, Instance location) {
+	private static SharedSlot removeFromMultiMap(Map<ResourceID, List<SharedSlot>> map, ResourceID location) {
 		List<SharedSlot> slotsForLocation = map.get(location);
 		
 		if (slotsForLocation == null) {
@@ -645,8 +629,8 @@ public class SlotSharingGroupAssignment {
 		}
 	}
 	
-	private static SharedSlot pollFromMultiMap(Map<Instance, List<SharedSlot>> map) {
-		Iterator<Map.Entry<Instance, List<SharedSlot>>> iter = map.entrySet().iterator();
+	private static SharedSlot pollFromMultiMap(Map<ResourceID, List<SharedSlot>> map) {
+		Iterator<Map.Entry<ResourceID, List<SharedSlot>>> iter = map.entrySet().iterator();
 		
 		while (iter.hasNext()) {
 			List<SharedSlot> slots = iter.next().getValue();
@@ -667,19 +651,19 @@ public class SlotSharingGroupAssignment {
 		return null;
 	}
 	
-	private static void removeSlotFromAllEntries(Map<AbstractID, Map<Instance, List<SharedSlot>>> availableSlots, 
-													SharedSlot slot)
+	private static void removeSlotFromAllEntries(
+			Map<AbstractID, Map<ResourceID, List<SharedSlot>>> availableSlots, SharedSlot slot)
 	{
-		final Instance instance = slot.getInstance();
+		final ResourceID taskManagerId = slot.getTaskManagerID();
 		
-		for (Map.Entry<AbstractID, Map<Instance, List<SharedSlot>>> entry : availableSlots.entrySet()) {
-			Map<Instance, List<SharedSlot>> map = entry.getValue();
+		for (Map.Entry<AbstractID, Map<ResourceID, List<SharedSlot>>> entry : availableSlots.entrySet()) {
+			Map<ResourceID, List<SharedSlot>> map = entry.getValue();
 
-			List<SharedSlot> list = map.get(instance);
+			List<SharedSlot> list = map.get(taskManagerId);
 			if (list != null) {
 				list.remove(slot);
 				if (list.isEmpty()) {
-					map.remove(instance);
+					map.remove(taskManagerId);
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
----------------------------------------------------------------------
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 fece894..c41f7bf 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.
  * 
- * <p>This class tracks the location and the shared slot for this set of tasks.</p>
+ * <p>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 class CoLocationConstraint {
 	 * @return True if the location has been assigned, false otherwise.
 	 */
 	public boolean isAssigned() {
-		return locationLocked;
+		return lockedLocation != null;
 	}
 
 	/**
@@ -89,7 +94,7 @@ public class CoLocationConstraint {
 	 *         false otherwise.
 	 */
 	public boolean isAssignedAndAlive() {
-		return locationLocked && sharedSlot.isAlive();
+		return lockedLocation != null && sharedSlot.isAlive();
 	}
 
 	/**
@@ -100,9 +105,9 @@ public class CoLocationConstraint {
 	 * @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 class CoLocationConstraint {
 	 *                                  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 @@ public class CoLocationConstraint {
 	 *                               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();
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
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 963fc4c..b481b55 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.Collections;
 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 akka.dispatch.Futures;
 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.executiongraph.ExecutionVertex;
 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<String, Set<Instance>> allInstancesByHost = new HashMap<String, Set<Instance>>();
 	
 	/** All instances that still have available resources */
-	private final Queue<Instance> instancesWithAvailableResources = new SetQueue<Instance>();
+	private final Map<ResourceID, Instance> instancesWithAvailableResources = new LinkedHashMap<>();
 	
 	/** All tasks pending to be scheduled */
 	private final Queue<QueuedTask> taskQueue = new ArrayDeque<QueuedTask>();
@@ -163,7 +166,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 
 		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 		
-		final Iterable<Instance> preferredLocations = vertex.getPreferredLocations();
+		final Iterable<TaskManagerLocation> preferredLocations = vertex.getPreferredLocations();
 		final boolean forceExternalLocation = vertex.isScheduleLocalOnly() &&
 									preferredLocations != null && preferredLocations.iterator().hasNext();
 	
@@ -222,7 +225,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 					
 					// our location preference is either determined by the location constraint, or by the
 					// vertex's preferred locations
-					final Iterable<Instance> locations;
+					final Iterable<TaskManagerLocation> locations;
 					final boolean localOnly;
 					if (constraint != null && constraint.isAssigned()) {
 						locations = Collections.singleton(constraint.getLocation());
@@ -341,7 +344,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
 	 */
 	protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex,
-											Iterable<Instance> requestedLocations,
+											Iterable<TaskManagerLocation> 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				
 				// 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 	 * @return A sub-slot for the given vertex, or {@code null}, if no slot is available.
 	 */
 	protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex,
-													Iterable<Instance> requestedLocations,
+													Iterable<TaskManagerLocation> requestedLocations,
 													SlotSharingGroupAssignment groupAssignment,
 													CoLocationConstraint constraint,
 													boolean localOnly)
@@ -422,7 +425,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 
 				// 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 	 *                           no locality preference exists.   
 	 * @param localOnly Flag to indicate whether only one of the exact local instances can be chosen.  
 	 */
-	private Pair<Instance, Locality> findInstance(Iterable<Instance> requestedLocations, boolean localOnly){
+	private Pair<Instance, Locality> findInstance(Iterable<TaskManagerLocation> 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 			return null;
 		}
 
-		Iterator<Instance> locations = requestedLocations == null ? null : requestedLocations.iterator();
+		Iterator<TaskManagerLocation> 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<Instance, Locality>(location, Locality.LOCAL);
+				TaskManagerLocation location = locations.next();
+				if (location != null) {
+					Instance instance = instancesWithAvailableResources.remove(location.getResourceID());
+					if (instance != null) {
+						return new ImmutablePair<Instance, Locality>(instance, Locality.LOCAL);
+					}
 				}
 			}
 			
@@ -492,14 +498,21 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				return null;
 			}
 			else {
-				Instance instanceToUse = this.instancesWithAvailableResources.poll();
-				return new ImmutablePair<Instance, Locality>(instanceToUse, Locality.NON_LOCAL);
+				// take the first instance from the instances with resources
+				Iterator<Instance> 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<Instance, Locality>(instanceToUse, Locality.UNCONSTRAINED);
+			Iterator<Instance> instances = instancesWithAvailableResources.values().iterator();
+			Instance instanceToUse = instances.next();
+			instances.remove();
+
+			return new ImmutablePair<>(instanceToUse, Locality.UNCONSTRAINED);
 		}
 	}
 	
@@ -570,7 +583,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				}
 			}
 			else {
-				this.instancesWithAvailableResources.add(instance);
+				this.instancesWithAvailableResources.put(instance.getResourceId(), instance);
 			}
 		}
 	}
@@ -643,11 +656,10 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 					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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 		}
 
 		allInstances.remove(instance);
-		instancesWithAvailableResources.remove(instance);
-		
+		instancesWithAvailableResources.remove(instance.getResourceId());
+
 		String instanceHostName = instance.getInstanceConnectionInfo().getHostname();
 		Set<Instance> instanceSet = allInstancesByHost.get(instanceHostName);
 		if (instanceSet != null) {
@@ -709,7 +721,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 		synchronized (globalLock) {
 			processNewlyAvailableInstances();
 
-			for (Instance instance : instancesWithAvailableResources) {
+			for (Instance instance : instancesWithAvailableResources.values()) {
 				count += instance.getNumberOfAvailableSlots();
 			}
 		}
@@ -781,9 +793,9 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 		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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	private static String getHostnamesFromInstances(Iterable<Instance> instances) {
+	private static String getHostnamesFromInstances(Iterable<TaskManagerLocation> 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();

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java
----------------------------------------------------------------------
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 31bd341..36e4072 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 @@ package org.apache.flink.runtime.jobmanager.scheduler;
 
 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";

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java
----------------------------------------------------------------------
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 0000000..ad9c784
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java
----------------------------------------------------------------------
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 5a0faa5..01d0654 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
@@ -108,7 +108,7 @@ public class TaskManagerLocation implements Comparable<TaskManagerLocation>, jav
 		}
 
 		this.stringRepresentation = String.format(
-				"TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort);
+				"%s @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort);
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 0c62c69..2a0ecc2 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
             }

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
----------------------------------------------------------------------
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 6a9b490..3947b17 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())
                 )
               )
             }

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
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 cf7cf58..d8bd6cb 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 class ExecutionGraphMetricsTest extends TestLogger {
 		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 class ExecutionGraphMetricsTest extends TestLogger {
 		assertNotNull(metric);
 		assertTrue(metric instanceof Gauge);
 
+		@SuppressWarnings("unchecked")
 		Gauge<Long> restartingTime = (Gauge<Long>) metric;
 
 		// check that the restarting time is 0 since it's the initial start

http://git-wip-us.apache.org/repos/asf/flink/blob/aaa474ad/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java
----------------------------------------------------------------------
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 91472ae..a1f3345 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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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 class VertexLocationConstraintTest {
 			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());


[4/8] flink git commit: [FLINK-4567] [runtime] Enhance SerializedThrowable to properly mimic Exception causes

Posted by se...@apache.org.
[FLINK-4567] [runtime] Enhance SerializedThrowable to properly mimic Exception causes


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/761d0a02
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/761d0a02
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/761d0a02

Branch: refs/heads/master
Commit: 761d0a02505c7eaef7a566f978145b187c89cbf8
Parents: c251efc
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 2 11:38:53 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/util/SerializedThrowable.java | 88 ++++++++++----------
 .../runtime/util/SerializedThrowableTest.java   | 40 ++++++++-
 2 files changed, 83 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/761d0a02/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
index a7739ef..4dea59c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
@@ -21,18 +21,19 @@ package org.apache.flink.runtime.util;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.InstantiationUtil;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.Serializable;
 import java.lang.ref.WeakReference;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * Utility class for dealing with user-defined Throwable types that are serialized (for
  * example during RPC/Actor communication), but cannot be resolved with the default
  * class loader.
- * <p>
- * This exception mimics the original exception with respect to message and stack trace,
+ * 
+ * <p>This exception mimics the original exception with respect to message and stack trace,
  * and contains the original exception in serialized form. The original exception
  * can be re-obtained by supplying the appropriate class loader.
  */
@@ -49,10 +50,6 @@ public class SerializedThrowable extends Exception implements Serializable {
 	/** The original stack trace, to be printed */
 	private final String fullStingifiedStackTrace;
 
-	/** A guaranteed serializable placeholder exception that will be used as
-	 * cause and to capture the original stack trace */
-	private final Exception placeholder;
-	
 	/** The original exception, not transported via serialization, 
 	 * because the class may not be part of the system class loader.
 	 * In addition, we make sure our cached references to not prevent
@@ -66,33 +63,43 @@ public class SerializedThrowable extends Exception implements Serializable {
 	 * @param exception The exception to serialize.
 	 */
 	public SerializedThrowable(Throwable exception) {
+		this(exception, new HashSet<Throwable>());
+	}
+
+	private SerializedThrowable(Throwable exception, Set<Throwable> alreadySeen) {
 		super(getMessageOrError(exception));
 
 		if (!(exception instanceof SerializedThrowable)) {
-			this.cachedException = new WeakReference<Throwable>(exception);
-			
-			this.originalErrorClassName = exception.getClass().getName();
-			this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception);
-			this.placeholder = new Exception(
-					"Serialized representation of " + originalErrorClassName + ": " + getMessage());
-			this.placeholder.setStackTrace(exception.getStackTrace());
-			initCause(this.placeholder);
-			
+			// serialize and memoize the original message
 			byte[] serialized;
 			try {
 				serialized = InstantiationUtil.serializeObject(exception);
 			}
 			catch (Throwable t) {
-				// could not serialize exception. send the stringified version instead
-				try {
-					serialized = InstantiationUtil.serializeObject(placeholder);
-				}
-				catch (IOException e) {
-					// this should really never happen, as we only serialize a a standard exception
-					throw new RuntimeException(e.getMessage(), e);
-				}
+				serialized = null;
 			}
 			this.serializedException = serialized;
+			this.cachedException = new WeakReference<Throwable>(exception);
+
+			// record the original exception's properties (name, stack prints)
+			this.originalErrorClassName = exception.getClass().getName();
+			this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception);
+
+			// mimic the original exception's stack trace
+			setStackTrace(exception.getStackTrace());
+
+			// mimic the original exception's cause
+			if (exception.getCause() == null) {
+				initCause(null);
+			}
+			else {
+				// exception causes may by cyclic, so we truncate the cycle when we find it 
+				if (alreadySeen.add(exception)) {
+					// we are not in a cycle, yet
+					initCause(new SerializedThrowable(exception.getCause(), alreadySeen));
+				}
+			}
+
 		}
 		else {
 			// copy from that serialized throwable
@@ -100,39 +107,37 @@ public class SerializedThrowable extends Exception implements Serializable {
 			this.serializedException = other.serializedException;
 			this.originalErrorClassName = other.originalErrorClassName;
 			this.fullStingifiedStackTrace = other.fullStingifiedStackTrace;
-			this.placeholder = other.placeholder;
 			this.cachedException = other.cachedException;
 		}
 	}
 
 	public Throwable deserializeError(ClassLoader classloader) {
+		if (serializedException == null) {
+			// failed to serialize the original exception
+			// return this SerializedThrowable as a stand in
+			return this;
+		}
+
 		Throwable cached = cachedException == null ? null : cachedException.get();
 		if (cached == null) {
 			try {
 				cached = InstantiationUtil.deserializeObject(serializedException, classloader);
 				cachedException = new WeakReference<Throwable>(cached);
 			}
-			catch (Exception e) {
-				return placeholder;
+			catch (Throwable t) {
+				// something went wrong
+				// return this SerializedThrowable as a stand in
+				return this;
 			}
 		}
 		return cached;
 	}
-	
-	public String getStrigifiedStackTrace() {
-		return fullStingifiedStackTrace;
-	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Override the behavior of Throwable
 	// ------------------------------------------------------------------------
 
 	@Override
-	public Throwable getCause() {
-		return placeholder;
-	}
-
-	@Override
 	public void printStackTrace(PrintStream s) {
 		s.print(fullStingifiedStackTrace);
 		s.flush();
@@ -150,15 +155,10 @@ public class SerializedThrowable extends Exception implements Serializable {
 		return (message != null) ? (originalErrorClassName + ": " + message) : originalErrorClassName;
 	}
 
-	@Override
-	public StackTraceElement[] getStackTrace() {
-		return placeholder.getStackTrace();
-	}
-
 	// ------------------------------------------------------------------------
 	//  Static utilities
 	// ------------------------------------------------------------------------
-	
+
 	public static Throwable get(Throwable serThrowable, ClassLoader loader) {
 		if (serThrowable instanceof SerializedThrowable) {
 			return ((SerializedThrowable)serThrowable).deserializeError(loader);
@@ -166,7 +166,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 			return serThrowable;
 		}
 	}
-	
+
 	private static String getMessageOrError(Throwable error) {
 		try {
 			return error.getMessage();

http://git-wip-us.apache.org/repos/asf/flink/blob/761d0a02/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
index 50efd52..4d57892 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
@@ -131,10 +131,48 @@ public class SerializedThrowableTest {
 			// deserialize the proper exception
 			Throwable deserialized = copy.deserializeError(loader); 
 			assertEquals(clazz, deserialized.getClass());
+
+			// deserialization with the wrong classloader does not lead to a failure
+			Throwable wronglyDeserialized = copy.deserializeError(getClass().getClassLoader());
+			assertEquals(ExceptionUtils.stringifyException(userException),
+					ExceptionUtils.stringifyException(wronglyDeserialized));
 		}
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
-	} 
+	}
+
+	@Test
+	public void testCauseChaining() {
+		Exception cause2 = new Exception("level2");
+		Exception cause1 = new Exception("level1", cause2);
+		Exception root = new Exception("level0", cause1);
+
+		SerializedThrowable st = new SerializedThrowable(root);
+
+		assertEquals("level0", st.getMessage());
+
+		assertNotNull(st.getCause());
+		assertEquals("level1", st.getCause().getMessage());
+
+		assertNotNull(st.getCause().getCause());
+		assertEquals("level2", st.getCause().getCause().getMessage());
+	}
+
+	@Test
+	public void testCyclicCauseChaining() {
+		Exception cause3 = new Exception("level3");
+		Exception cause2 = new Exception("level2", cause3);
+		Exception cause1 = new Exception("level1", cause2);
+		Exception root = new Exception("level0", cause1);
+
+		// introduce a cyclic reference
+		cause3.initCause(cause1);
+
+		SerializedThrowable st = new SerializedThrowable(root);
+
+		assertArrayEquals(root.getStackTrace(), st.getStackTrace());
+		assertEquals(ExceptionUtils.stringifyException(root), ExceptionUtils.stringifyException(st));
+	}
 }


[6/8] flink git commit: [FLINK-4490] [distributed coordination] (part 3) Rename methods on 'Instance' to have more intuitive names

Posted by se...@apache.org.
[FLINK-4490] [distributed coordination] (part 3) Rename methods on 'Instance' to have more intuitive names

getResourceID() --> getTaskManagerID()
getInstanceConnectionInfo() --> getTaskManagerLocation()


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/eac6088a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/eac6088a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/eac6088a

Branch: refs/heads/master
Commit: eac6088a75e813a015b778f4cfc4cce0cf2a53ce
Parents: aaa474a
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Aug 31 13:59:01 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../handlers/TaskManagersHandler.java           |   2 +-
 ...PartialInputChannelDeploymentDescriptor.java |  21 +-
 .../apache/flink/runtime/instance/Instance.java |  28 +-
 .../flink/runtime/instance/InstanceManager.java |  20 +-
 .../runtime/jobmanager/scheduler/Scheduler.java |  18 +-
 .../flink/runtime/jobmanager/JobManager.scala   |   3 +-
 .../ExecutionGraphMetricsTest.java              | 404 ++++++++++---------
 .../executiongraph/ExecutionGraphTestUtils.java |   2 +-
 .../TerminalStateDeadlockTest.java              |   3 +-
 .../runtime/instance/InstanceManagerTest.java   |  31 +-
 .../flink/runtime/instance/InstanceTest.java    |   6 +-
 .../flink/runtime/instance/SharedSlotsTest.java |  24 +-
 .../flink/runtime/instance/SimpleSlotTest.java  |   2 +-
 .../partition/SpilledSubpartitionViewTest.java  |   4 +-
 .../scheduler/CoLocationConstraintTest.java     |   6 +-
 .../ScheduleWithCoLocationHintTest.java         |  22 +-
 .../scheduler/SchedulerSlotSharingTest.java     |  30 +-
 .../scheduler/SchedulerTestUtils.java           |   5 +-
 .../scheduler/SlotAllocationFutureTest.java     |  12 +-
 19 files changed, 319 insertions(+), 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
----------------------------------------------------------------------
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 b60cd10..b5e9088 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 class TaskManagersHandler implements RequestHandler {
 					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());

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java
----------------------------------------------------------------------
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 e1391a4..0eac39d 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
@@ -44,7 +44,7 @@ public class PartialInputChannelDeploymentDescriptor {
 	private final ResultPartitionID partitionID;
 
 	/** The partition connection info. */
-	private final TaskManagerLocation partitionConnectionInfo;
+	private final TaskManagerLocation partitionTaskManagerLocation;
 
 	/** The partition connection index. */
 	private final int partitionConnectionIndex;
@@ -52,12 +52,12 @@ public class PartialInputChannelDeploymentDescriptor {
 	public PartialInputChannelDeploymentDescriptor(
 			IntermediateDataSetID resultId,
 			ResultPartitionID partitionID,
-			TaskManagerLocation partitionConnectionInfo,
+			TaskManagerLocation partitionTaskManagerLocation,
 			int partitionConnectionIndex) {
 
 		this.resultId = checkNotNull(resultId);
 		this.partitionID = checkNotNull(partitionID);
-		this.partitionConnectionInfo = checkNotNull(partitionConnectionInfo);
+		this.partitionTaskManagerLocation = checkNotNull(partitionTaskManagerLocation);
 		this.partitionConnectionIndex = partitionConnectionIndex;
 	}
 
@@ -66,23 +66,20 @@ public class PartialInputChannelDeploymentDescriptor {
 	 *
 	 * @see InputChannelDeploymentDescriptor
 	 */
-	public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor(
-			Execution consumerExecution) {
+	public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor(Execution consumerExecution) {
+		checkNotNull(consumerExecution, "consumerExecution");
 
-		checkNotNull(consumerExecution, "Consumer execution null");
-
-		TaskManagerLocation consumerConnectionInfo = consumerExecution.getAssignedResourceLocation();
-
-		checkNotNull(consumerConnectionInfo, "Consumer connection info null");
+		TaskManagerLocation consumerLocation = consumerExecution.getAssignedResourceLocation();
+		checkNotNull(consumerLocation, "Consumer connection info null");
 
 		final ResultPartitionLocation partitionLocation;
 
-		if (consumerConnectionInfo.equals(partitionConnectionInfo)) {
+		if (consumerLocation.equals(partitionTaskManagerLocation)) {
 			partitionLocation = ResultPartitionLocation.createLocal();
 		}
 		else {
 			partitionLocation = ResultPartitionLocation.createRemote(
-					new ConnectionID(partitionConnectionInfo, partitionConnectionIndex));
+					new ConnectionID(partitionTaskManagerLocation, partitionConnectionIndex));
 		}
 
 		return new InputChannelDeploymentDescriptor(partitionID, partitionLocation);

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
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 fe46895..4a8139b 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,14 +52,11 @@ 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 identifying the taskManager. */
 	private final InstanceID instanceId;
 
@@ -90,22 +87,19 @@ 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 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,
 			InstanceID id,
 			HardwareDescription resources,
 			int numberOfSlots) {
 		this.actorGateway = actorGateway;
-		this.connectionInfo = connectionInfo;
-		this.resourceId = resourceId;
+		this.location = location;
 		this.instanceId = id;
 		this.resources = resources;
 		this.numberOfSlots = numberOfSlots;
@@ -120,8 +114,8 @@ public class Instance implements SlotOwner {
 	// Properties
 	// --------------------------------------------------------------------------------------------
 
-	public ResourceID getResourceId() {
-		return resourceId;
+	public ResourceID getTaskManagerID() {
+		return location.getResourceID();
 	}
 
 	public InstanceID getId() {
@@ -246,7 +240,7 @@ public class Instance implements SlotOwner {
 				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 +278,7 @@ public class Instance implements SlotOwner {
 			}
 			else {
 				SharedSlot slot = new SharedSlot(
-						jobID, this, connectionInfo, nextSlot, actorGateway, sharingGroupAssignment);
+						jobID, this, location, nextSlot, actorGateway, sharingGroupAssignment);
 				allocatedSlots.add(slot);
 				return slot;
 			}
@@ -355,8 +349,8 @@ public class Instance implements SlotOwner {
 		return actorGateway;
 	}
 
-	public TaskManagerLocation getInstanceConnectionInfo() {
-		return connectionInfo;
+	public TaskManagerLocation getTaskManagerLocation() {
+		return location;
 	}
 
 	public int getNumberOfAvailableSlots() {
@@ -405,7 +399,7 @@ public class Instance implements SlotOwner {
 
 	@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"));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
----------------------------------------------------------------------
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 e7a4537..0c7e187 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
@@ -138,8 +138,7 @@ public class InstanceManager {
 	 * for the job execution.
 	 *
 	 * @param taskManager ActorRef to the TaskManager which wants to be registered
-	 * @param resourceID The resource id of the TaskManager
-	 * @param connectionInfo ConnectionInfo of the TaskManager
+	 * @param taskManagerLocation Location info of the TaskManager
 	 * @param resources Hardware description of the TaskManager
 	 * @param numberOfSlots Number of available slots on the TaskManager
 	 * @param leaderSessionID The current leader session ID of the JobManager
@@ -147,12 +146,12 @@ public class InstanceManager {
 	 */
 	public InstanceID registerTaskManager(
 			ActorRef taskManager,
-			ResourceID resourceID,
-			TaskManagerLocation connectionInfo,
+			TaskManagerLocation taskManagerLocation,
 			HardwareDescription resources,
 			int numberOfSlots,
-			UUID leaderSessionID){
-		synchronized(this.lock){
+			UUID leaderSessionID) {
+		
+		synchronized (this.lock) {
 			if (this.isShutdown) {
 				throw new IllegalStateException("InstanceManager is shut down.");
 			}
@@ -174,12 +173,11 @@ public class InstanceManager {
 
 			InstanceID instanceID = new InstanceID();
 
-			Instance host = new Instance(actorGateway, connectionInfo, resourceID, instanceID,
-				resources, numberOfSlots);
+			Instance host = new Instance(actorGateway, taskManagerLocation, instanceID, resources, numberOfSlots);
 
 			registeredHostsById.put(instanceID, host);
 			registeredHostsByConnection.put(taskManager, host);
-			registeredHostsByResource.put(resourceID, host);
+			registeredHostsByResource.put(taskManagerLocation.getResourceID(), host);
 
 			totalNumberOfAliveTaskSlots += numberOfSlots;
 
@@ -187,7 +185,7 @@ public class InstanceManager {
 				LOG.info(String.format("Registered TaskManager at %s (%s) as %s. " +
 								"Current number of registered hosts is %d. " +
 								"Current number of alive task slots is %d.",
-						connectionInfo.getHostname(),
+						taskManagerLocation.getHostname(),
 						taskManager.path(),
 						instanceID,
 						registeredHostsById.size(),
@@ -217,7 +215,7 @@ public class InstanceManager {
 
 			registeredHostsByConnection.remove(host);
 			registeredHostsById.remove(instance.getId());
-			registeredHostsByResource.remove(instance.getResourceId());
+			registeredHostsByResource.remove(instance.getTaskManagerID());
 
 			if (terminated) {
 				deadHosts.add(instance.getActorGateway().actor());

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
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 b481b55..734972d 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				
 				// 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 
 				// 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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 		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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				}
 			}
 			else {
-				this.instancesWithAvailableResources.put(instance.getResourceId(), instance);
+				this.instancesWithAvailableResources.put(instance.getTaskManagerID(), instance);
 			}
 		}
 	}
@@ -649,7 +649,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				instance.setSlotAvailabilityListener(this);
 				
 				// store the instance in the by-host-lookup
-				String instanceHostName = instance.getInstanceConnectionInfo().getHostname();
+				String instanceHostName = instance.getTaskManagerLocation().getHostname();
 				Set<Instance> instanceSet = allInstancesByHost.get(instanceHostName);
 				if (instanceSet == null) {
 					instanceSet = new HashSet<Instance>();
@@ -658,7 +658,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 				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 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 		}
 
 		allInstances.remove(instance);
-		instancesWithAvailableResources.remove(instance.getResourceId());
+		instancesWithAvailableResources.remove(instance.getTaskManagerID());
 
-		String instanceHostName = instance.getInstanceConnectionInfo().getHostname();
+		String instanceHostName = instance.getTaskManagerLocation().getHostname();
 		Set<Instance> instanceSet = allInstancesByHost.get(instanceHostName);
 		if (instanceSet != null) {
 			instanceSet.remove(instance);
@@ -795,7 +795,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 
 			while ((instance = newlyAvailableInstances.poll()) != null) {
 				if (instance.hasResourcesAvailable()) {
-					instancesWithAvailableResources.put(instance.getResourceId(), instance);
+					instancesWithAvailableResources.put(instance.getTaskManagerID(), instance);
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 2a0ecc2..88af604 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))
@@ -425,7 +425,6 @@ class JobManager(
         try {
           val instanceID = instanceManager.registerTaskManager(
             taskManager,
-            resourceId,
             connectionInfo,
             hardwareInformation,
             numberOfSlots,

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
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 d8bd6cb..d5520fd 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.MetricGroup;
 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.metrics.groups.JobManagerMetricGroup;
 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.Collections;
 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$.<Object>successful(Messages.getAcknowledge()));
-
-		TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy();
-
-		ExecutionGraph executionGraph = new ExecutionGraph(
-			ExecutionContext$.MODULE$.fromExecutor(new ForkJoinPool()),
-			jobGraph.getJobID(),
-			jobGraph.getName(),
-			jobConfig,
-			new SerializedValue<ExecutionConfig>(null),
-			timeout,
-			testingRestartStrategy,
-			Collections.<BlobKey>emptyList(),
-			Collections.<URL>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<Long> restartingTime = (Gauge<Long>) 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<ExecutionAttemptID> 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$.<Object>successful(Messages.getAcknowledge()));
+
+			TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy();
+
+			ExecutionGraph executionGraph = new ExecutionGraph(
+				ExecutionContext$.MODULE$.fromExecutor(executor),
+				jobGraph.getJobID(),
+				jobGraph.getName(),
+				jobConfig,
+				new SerializedValue<ExecutionConfig>(null),
+				timeout,
+				testingRestartStrategy,
+				Collections.<BlobKey>emptyList(),
+				Collections.<URL>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<Long> restartingTime = (Gauge<Long>) 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<ExecutionAttemptID> 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();
 		}
 
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
----------------------------------------------------------------------
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 cddb6cb..df47c3a 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
@@ -113,7 +113,7 @@ public class ExecutionGraphTestUtils {
 		InetAddress address = InetAddress.getByName("127.0.0.1");
 		TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001);
 
-		return new Instance(gateway, connection, resourceID, new InstanceID(), hardwareDescription, numberOfSlots);
+		return new Instance(gateway, connection, new InstanceID(), hardwareDescription, numberOfSlots);
 	}
 
 	@SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
----------------------------------------------------------------------
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 a71faf6..870ae05 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
@@ -84,8 +84,7 @@ public class TerminalStateDeadlockTest {
 			TaskManagerLocation ci = new TaskManagerLocation(resourceId, address, 12345);
 				
 			HardwareDescription resources = new HardwareDescription(4, 4000000, 3000000, 2000000);
-			Instance instance = new Instance(DummyActorGateway.INSTANCE, ci,
-				resourceId, new InstanceID(), resources, 4);
+			Instance instance = new Instance(DummyActorGateway.INSTANCE, ci, new InstanceID(), resources, 4);
 
 			this.resource = instance.allocateSimpleSlot(new JobID());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
----------------------------------------------------------------------
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 f1ed960..f3747c8 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
@@ -87,12 +87,9 @@ public class InstanceManagerTest{
 			final JavaTestKit probe2 = new JavaTestKit(system);
 			final JavaTestKit probe3 = new JavaTestKit(system);
 
-			cm.registerTaskManager(probe1.getRef(), resID1,
-				ici1, hardwareDescription, 1, leaderSessionID);
-			cm.registerTaskManager(probe2.getRef(), resID2,
-				ici2, hardwareDescription, 2, leaderSessionID);
-			cm.registerTaskManager(probe3.getRef(), resID3,
-				ici3, hardwareDescription, 5, leaderSessionID);
+			cm.registerTaskManager(probe1.getRef(), ici1, hardwareDescription, 1, leaderSessionID);
+			cm.registerTaskManager(probe2.getRef(), ici2, hardwareDescription, 2, leaderSessionID);
+			cm.registerTaskManager(probe3.getRef(), ici3, hardwareDescription, 5, leaderSessionID);
 
 			assertEquals(3, cm.getNumberOfRegisteredTaskManagers());
 			assertEquals(8, cm.getTotalNumberOfSlots());
@@ -102,7 +99,7 @@ public class InstanceManagerTest{
 					HashSet<TaskManagerLocation>();
 
 			for(Instance instance: instances){
-				taskManagerLocations.add(instance.getInstanceConnectionInfo());
+				taskManagerLocations.add(instance.getTaskManagerLocation());
 			}
 
 			assertTrue(taskManagerLocations.contains(ici1));
@@ -133,14 +130,13 @@ public class InstanceManagerTest{
 			TaskManagerLocation ici = new TaskManagerLocation(resID1, address, dataPort);
 
 			JavaTestKit probe = new JavaTestKit(system);
-			cm.registerTaskManager(probe.getRef(), resID1,
-				ici, resources, 1, leaderSessionID);
+			cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID);
 
 			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
 			assertEquals(1, cm.getTotalNumberOfSlots());
 
 			try {
-				cm.registerTaskManager(probe.getRef(), resID2, ici, resources, 1, leaderSessionID);
+				cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID);
 			} catch (Exception e) {
 				// good
 			}
@@ -182,12 +178,12 @@ public class InstanceManagerTest{
 			JavaTestKit probe2 = new JavaTestKit(system);
 			JavaTestKit probe3 = new JavaTestKit(system);
 
-			InstanceID instanceID1 = cm.registerTaskManager(probe1.getRef(), resID1,
-				ici1, hardwareDescription, 1, leaderSessionID);
-			InstanceID instanceID2 = cm.registerTaskManager(probe2.getRef(), resID2,
-				ici2, hardwareDescription, 1, leaderSessionID);
-			InstanceID instanceID3 = cm.registerTaskManager(probe3.getRef(), resID3,
-				ici3, hardwareDescription, 1, leaderSessionID);
+			InstanceID instanceID1 = cm.registerTaskManager(
+					probe1.getRef(), ici1, hardwareDescription, 1, leaderSessionID);
+			InstanceID instanceID2 = cm.registerTaskManager(
+					probe2.getRef(), ici2, hardwareDescription, 1, leaderSessionID);
+			InstanceID instanceID3 = cm.registerTaskManager(
+					probe3.getRef(), ici3, hardwareDescription, 1, leaderSessionID);
 
 			// report some immediate heart beats
 			assertTrue(cm.reportHeartBeat(instanceID1, new byte[] {}));
@@ -241,8 +237,7 @@ public class InstanceManagerTest{
 				TaskManagerLocation ici = new TaskManagerLocation(resID, address, 20000);
 
 				JavaTestKit probe = new JavaTestKit(system);
-				cm.registerTaskManager(probe.getRef(), resID,
-					ici, resources, 1, leaderSessionID);
+				cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID);
 				fail("Should raise exception in shutdown state");
 			}
 			catch (IllegalStateException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
----------------------------------------------------------------------
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 82d3723..aee62fd 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
@@ -42,7 +42,7 @@ public class InstanceTest {
 			TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001);
 
 			Instance instance = new Instance(DummyActorGateway.INSTANCE, connection,
-					resourceID, new InstanceID(), hardwareDescription, 4);
+					new InstanceID(), hardwareDescription, 4);
 
 			assertEquals(4, instance.getTotalNumberOfSlots());
 			assertEquals(4, instance.getNumberOfAvailableSlots());
@@ -105,7 +105,7 @@ public class InstanceTest {
 			TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001);
 
 			Instance instance = new Instance(DummyActorGateway.INSTANCE, connection,
-					resourceID, new InstanceID(), hardwareDescription, 3);
+					new InstanceID(), hardwareDescription, 3);
 
 			assertEquals(3, instance.getNumberOfAvailableSlots());
 
@@ -137,7 +137,7 @@ public class InstanceTest {
 			TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001);
 
 			Instance instance = new Instance(DummyActorGateway.INSTANCE, connection,
-					resourceID, new InstanceID(), hardwareDescription, 3);
+					new InstanceID(), hardwareDescription, 3);
 
 			assertEquals(3, instance.getNumberOfAvailableSlots());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java
----------------------------------------------------------------------
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 2c40e89..0edef5e 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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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.<TaskManagerLocation>emptySet());
 			
@@ -475,7 +475,7 @@ public class SharedSlotsTest {
 			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 class SharedSlotsTest {
 			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());

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
----------------------------------------------------------------------
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 82c2a74..c690d36 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
@@ -150,7 +150,7 @@ public class SimpleSlotTest {
 		TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001);
 
 		Instance instance = new Instance(DummyActorGateway.INSTANCE, connection,
-				resourceID, new InstanceID(), hardwareDescription, 1);
+				new InstanceID(), hardwareDescription, 1);
 		return instance.allocateSimpleSlot(new JobID());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java
----------------------------------------------------------------------
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 fff7bc6..5722cac 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.runner.RunWith;
 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 class SpilledSubpartitionViewTest {
 				}
 			}
 
-			final List<Future<Boolean>> results = Lists.newArrayList();
+			final List<Future<Boolean>> results = new ArrayList<>();
 
 			// Submit the consuming tasks
 			for (ResultSubpartitionView view : readers) {

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java
----------------------------------------------------------------------
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 3bd4368..1344aef 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 class CoLocationConstraintTest {
 			// 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 class CoLocationConstraintTest {
 			// 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 class CoLocationConstraintTest {
 
 			assertTrue(constraint.isAssigned());
 			assertTrue(constraint.isAssignedAndAlive());
-			assertEquals(instance2, constraint.getLocation());
+			assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation());
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
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 5b7d18a..eab4fea 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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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 class ScheduleWithCoLocationHintTest {
 			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);

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
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 a683834..fd0523b 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 class SchedulerSlotSharingTest {
 			
 			// 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 class SchedulerSlotSharingTest {
 			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 class SchedulerSlotSharingTest {
 			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 class SchedulerSlotSharingTest {
 			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 class SchedulerSlotSharingTest {
 			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());

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
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 eef27a8..d040ec4 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
@@ -69,8 +69,7 @@ public class SchedulerTestUtils {
 		final long GB = 1024L*1024*1024;
 		HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB);
 		
-		return new Instance(DummyActorGateway.INSTANCE, ci, resourceID,
-			new InstanceID(), resources, numSlots);
+		return new Instance(DummyActorGateway.INSTANCE, ci, new InstanceID(), resources, numSlots);
 	}
 	
 	
@@ -88,7 +87,7 @@ public class SchedulerTestUtils {
 	public static Execution getTestVertex(Instance... preferredInstances) {
 		List<TaskManagerLocation> locations = new ArrayList<>(preferredInstances.length);
 		for (Instance i : preferredInstances) {
-			locations.add(i.getInstanceConnectionInfo());
+			locations.add(i.getTaskManagerLocation());
 		}
 		return getTestVertex(locations);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/eac6088a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
----------------------------------------------------------------------
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 d9c100c..ea0d2cc 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 class SlotAllocationFutureTest {
 			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 class SlotAllocationFutureTest {
 				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 class SlotAllocationFutureTest {
 				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 class SlotAllocationFutureTest {
 				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 class SlotAllocationFutureTest {
 				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);


[8/8] flink git commit: [FLINK-4490] [distributed coordination] (part 1) Change InstanceConnectionInfo to TaskManagerLocation

Posted by se...@apache.org.
[FLINK-4490] [distributed coordination] (part 1) Change InstanceConnectionInfo to TaskManagerLocation

This adds the ResourceId to the TaskManagerLocation


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/34cda87a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/34cda87a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/34cda87a

Branch: refs/heads/master
Commit: 34cda87a6e831c6ea62f1a79df08d1efac718d03
Parents: e227b10
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Aug 29 16:58:31 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../clusterframework/MesosTaskManager.scala     |   7 +-
 .../handlers/JobExceptionsHandler.java          |   4 +-
 .../handlers/JobVertexDetailsHandler.java       |   4 +-
 .../handlers/JobVertexTaskManagersHandler.java  |   4 +-
 .../SubtaskExecutionAttemptDetailsHandler.java  |   4 +-
 .../SubtasksAllAccumulatorsHandler.java         |   4 +-
 .../handlers/SubtasksTimesHandler.java          |   4 +-
 ...PartialInputChannelDeploymentDescriptor.java |  10 +-
 .../flink/runtime/executiongraph/Execution.java |   8 +-
 .../runtime/executiongraph/ExecutionVertex.java |   4 +-
 .../apache/flink/runtime/instance/Instance.java |   7 +-
 .../instance/InstanceConnectionInfo.java        | 267 -------------------
 .../flink/runtime/instance/InstanceManager.java |   3 +-
 .../flink/runtime/io/network/ConnectionID.java  |   4 +-
 .../taskmanager/TaskManagerLocation.java        | 257 ++++++++++++++++++
 .../runtime/messages/RegistrationMessages.scala |  12 +-
 .../flink/runtime/taskmanager/TaskManager.scala |  15 +-
 .../testingUtils/TestingTaskManager.scala       |  33 ++-
 .../ExecutionGraphMetricsTest.java              |   8 +-
 .../executiongraph/ExecutionGraphTestUtils.java |   7 +-
 .../TerminalStateDeadlockTest.java              |   7 +-
 .../VertexLocationConstraintTest.java           |   6 +-
 .../instance/InstanceConnectionInfoTest.java    | 200 --------------
 .../runtime/instance/InstanceManagerTest.java   |  64 +++--
 .../flink/runtime/instance/InstanceTest.java    |  16 +-
 .../flink/runtime/instance/SimpleSlotTest.java  |   6 +-
 .../scheduler/SchedulerTestUtils.java           |  25 +-
 .../resourcemanager/ResourceManagerITCase.java  |   5 +-
 ...askManagerComponentsStartupShutdownTest.java |  12 +-
 .../taskmanager/TaskManagerLocationTest.java    | 201 ++++++++++++++
 .../jobmanager/JobManagerRegistrationTest.scala |  14 +-
 .../flink/yarn/TestingYarnTaskManager.scala     |  20 +-
 .../org/apache/flink/yarn/YarnTaskManager.scala |   7 +-
 33 files changed, 618 insertions(+), 631 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
----------------------------------------------------------------------
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 d8b6775..19b0c62 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,
+    taskManagerLocation: TaskManagerLocation,
     memoryManager: MemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
@@ -41,7 +40,7 @@ class MesosTaskManager(
   extends TaskManager(
     config,
     resourceID,
-    connectionInfo,
+    taskManagerLocation,
     memoryManager,
     ioManager,
     network,

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
----------------------------------------------------------------------
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 7b6a361..ce154e3 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 @@ package org.apache.flink.runtime.webmonitor.handlers;
 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 class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
 					break;
 				}
 
-				InstanceConnectionInfo location = task.getCurrentAssignedResourceLocation();
+				TaskManagerLocation location = task.getCurrentAssignedResourceLocation();
 				String locationString = location != null ?
 						location.getFQDNHostname() + ':' + location.dataPort() : "(unassigned)";
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java
----------------------------------------------------------------------
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 d4e885e..813ecb8 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.accumulators.AccumulatorRegistry;
 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 class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
 		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);

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java
----------------------------------------------------------------------
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 befc0bf..cbdb87f 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.accumulators.AccumulatorRegistry;
 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 class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandle
 		Map<String, List<ExecutionVertex>> 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<ExecutionVertex> vertices = taskManagerVertices.get(taskManager);

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java
----------------------------------------------------------------------
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 3d80b23..a1e6d0e 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.api.common.accumulators.LongCounter;
 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 class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp
 		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);

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java
----------------------------------------------------------------------
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 6d9ce3a..780bd4b 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 com.fasterxml.jackson.core.JsonGenerator;
 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 class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHand
 		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();

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java
----------------------------------------------------------------------
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 03d40dc..9e6276d 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 com.fasterxml.jackson.core.JsonGenerator;
 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 class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
 			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);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java
----------------------------------------------------------------------
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 a3cfcd9..e1391a4 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 @@ package org.apache.flink.runtime.deployment;
 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 class PartialInputChannelDeploymentDescriptor {
 
 		checkNotNull(consumerExecution, "Consumer execution null");
 
-		InstanceConnectionInfo consumerConnectionInfo = consumerExecution.getAssignedResourceLocation();
+		TaskManagerLocation consumerConnectionInfo = consumerExecution.getAssignedResourceLocation();
 
 		checkNotNull(consumerConnectionInfo, "Consumer connection info null");
 
@@ -107,7 +107,7 @@ public class PartialInputChannelDeploymentDescriptor {
 		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(

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
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 efddecc..197999c 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.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.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;
@@ -133,7 +133,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
 
 	private ChainedStateHandle<StreamStateHandle> chainedStateHandle;
 
@@ -147,7 +147,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<String, Accumulator<?, ?>> userAccumulators;
@@ -202,7 +202,7 @@ public class Execution {
 		return assignedResource;
 	}
 
-	public InstanceConnectionInfo getAssignedResourceLocation() {
+	public TaskManagerLocation getAssignedResourceLocation() {
 		return assignedResourceLocation;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index b215394..e5a115a 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.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.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;
@@ -228,7 +228,7 @@ public class ExecutionVertex {
 		return currentExecution.getAssignedResource();
 	}
 
-	public InstanceConnectionInfo getCurrentAssignedResourceLocation() {
+	public TaskManagerLocation getCurrentAssignedResourceLocation() {
 		return currentExecution.getAssignedResourceLocation();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
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 119f060..598b32b 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 java.util.Set;
 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 class Instance {
 		return actorGateway;
 	}
 
-	public InstanceConnectionInfo getInstanceConnectionInfo() {
+	public TaskManagerLocation getInstanceConnectionInfo() {
 		return connectionInfo;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
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/instance/InstanceConnectionInfo.java
deleted file mode 100644
index 2830f04..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
+++ /dev/null
@@ -1,267 +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.instance;
-
-import java.io.IOException;
-import java.io.Serializable;
-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.util.NetUtils;
-import org.apache.flink.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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<InstanceConnectionInfo>, 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;
-
-	/**
-	 * 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 hostname, derived from the fully qualified host name.
-	 */
-	private String hostName;
-
-
-	/**
-	 * Constructs a new instance connection info object. The constructor will attempt to retrieve the instance's
-	 * host name and domain name through the operating system's lookup mechanisms.
-	 * 
-	 * @param inetAddress
-	 *        the network address the instance's task manager binds its sockets to
-	 * @param dataPort
-	 *        the port instance's task manager expects to receive transfer envelopes on
-	 */
-	public InstanceConnectionInfo(InetAddress inetAddress, int dataPort) {
-		if (inetAddress == null) {
-			throw new IllegalArgumentException("Argument inetAddress must not be null");
-		}
-
-		// -1 indicates a local instance connection info
-		if (dataPort != -1 && dataPort <= 0) {
-			throw new IllegalArgumentException("Argument dataPort must be greater than zero");
-		}
-
-		this.dataPort = dataPort;
-		this.inetAddress = inetAddress;
-		
-		// get FQDN hostname on this TaskManager.
-		try {
-			this.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();
-		}
-
-		if (this.fqdnHostName.equals(this.inetAddress.getHostAddress())) {
-			// this happens when the name lookup fails, either due to an exception,
-			// or because no hostname can be found for the address
-			// take IP textual representation
-			this.hostName = this.fqdnHostName;
-			LOG.warn("No hostname could be resolved for the IP address {}, using IP address as host name. "
-					+ "Local input split assignment (such as for HDFS files) may be impacted.",
-					this.inetAddress.getHostAddress());
-		}
-		else {
-			this.hostName = NetUtils.getHostnameFromFQDN(this.fqdnHostName);
-		}
-	}
-
-	/**
-	 * Constructs an empty object.
-	 */
-	public InstanceConnectionInfo() {}
-
-
-	/**
-	 * Returns the port instance's task manager expects to receive transfer envelopes on.
-	 * 
-	 * @return the port instance's task manager expects to receive transfer envelopes on
-	 */
-	public int dataPort() {
-		return this.dataPort;
-	}
-
-	/**
-	 * Returns the network address the instance's task manager binds its sockets to.
-	 * 
-	 * @return the network address the instance's task manager binds its sockets to
-	 */
-	public InetAddress address() {
-		return this.inetAddress;
-	}
-
-	/**
-	 * Returns the fully-qualified domain name the TaskManager. If the name could not be
-	 * determined, the return value will be a textual representation of the TaskManager's IP address.
-	 * 
-	 * @return The fully-qualified domain name of the TaskManager.
-	 */
-	public String getFQDNHostname() {
-		return this.fqdnHostName;
-	}
-
-	/**
-	 * Gets the hostname of the TaskManager. The hostname derives from the fully qualified
-	 * domain name (FQDN, see {@link #getFQDNHostname()}):
-	 * <ul>
-	 *     <li>If the FQDN is the textual IP address, then the hostname is also the IP address</li>
-	 *     <li>If the FQDN has only one segment (such as "localhost", or "host17"), then this is
-	 *         used as the hostname.</li>
-	 *     <li>If the FQDN has multiple segments (such as "worker3.subgroup.company.net"), then the first
-	 *         segment (here "worker3") will be used as the hostname.</li>
-	 * </ul>
-	 *
-	 * @return The hostname of the TaskManager.
-	 */
-	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 + ")";
-	}
-
-	@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 {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return this.inetAddress.hashCode() +
-				17*dataPort;
-	}
-
-	@Override
-	public int compareTo(InstanceConnectionInfo o) {
-		// decide based on address first
-		byte[] thisAddress = this.inetAddress.getAddress();
-		byte[] otherAddress = o.inetAddress.getAddress();
-		
-		if (thisAddress.length < otherAddress.length) {
-			return -1;
-		} else if (thisAddress.length > otherAddress.length) {
-			return 1;
-		} else {
-			for (int i = 0; i < thisAddress.length; i++) {
-				byte tb = thisAddress[i];
-				byte ob = otherAddress[i];
-				if (tb < ob) {
-					return -1;
-				} else if (tb > ob) {
-					return 1;
-				}
-			}
-		}
-		
-		// addresses are identical, decide based on ports.
-		if (this.dataPort < o.dataPort) {
-			return -1;
-		} else if (this.dataPort > o.dataPort) {
-			return 1;
-		} else {
-			return 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
----------------------------------------------------------------------
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 0d0d4c7..e7a4537 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 java.util.UUID;
 
 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 class InstanceManager {
 	public InstanceID registerTaskManager(
 			ActorRef taskManager,
 			ResourceID resourceID,
-			InstanceConnectionInfo connectionInfo,
+			TaskManagerLocation connectionInfo,
 			HardwareDescription resources,
 			int numberOfSlots,
 			UUID leaderSessionID){

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java
----------------------------------------------------------------------
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 0569dae..cc2a19d 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;
@@ -43,7 +43,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);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5a0faa5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java
@@ -0,0 +1,257 @@
+/*
+ * 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.taskmanager;
+
+import java.net.InetAddress;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.util.NetUtils;
+
+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 TaskManagerLocation implements Comparable<TaskManagerLocation>, java.io.Serializable {
+
+	private static final long serialVersionUID = -8254407801276350716L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerLocation.class);
+
+	// ------------------------------------------------------------------------
+
+	/** 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 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
+	 * host name and domain name through the operating system's lookup mechanisms.
+	 * 
+	 * @param inetAddress
+	 *        the network address the instance's task manager binds its sockets to
+	 * @param dataPort
+	 *        the port instance's task manager expects to receive transfer envelopes on
+	 */
+	public TaskManagerLocation(ResourceID resourceID, InetAddress inetAddress, int dataPort) {
+		// -1 indicates a local instance connection info
+		checkArgument(dataPort > 0 || dataPort == -1, "dataPort must be > 0, or -1 (local)");
+
+		this.resourceID = checkNotNull(resourceID);
+		this.inetAddress = checkNotNull(inetAddress);
+		this.dataPort = dataPort;
+
+		// get FQDN hostname on this TaskManager.
+		String fqdnHostName;
+		try {
+			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);
+			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,
+			// or because no hostname can be found for the address
+			// take IP textual representation
+			this.hostName = this.fqdnHostName;
+			LOG.warn("No hostname could be resolved for the IP address {}, using IP address as host name. "
+					+ "Local input split assignment (such as for HDFS files) may be impacted.",
+					this.inetAddress.getHostAddress());
+		}
+		else {
+			this.hostName = NetUtils.getHostnameFromFQDN(this.fqdnHostName);
+		}
+
+		this.stringRepresentation = String.format(
+				"TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Getters
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the ID of the resource in which the TaskManager is started. The format of this depends
+	 * on how the TaskManager is started:
+	 * <ul>
+	 *     <li>If the TaskManager is started via YARN, this is the YARN container ID.</li>
+	 *     <li>If the TaskManager is started via Mesos, this is the Mesos container ID.</li>
+	 *     <li>If the TaskManager is started in standalone mode, or via a MiniCluster, this is a random ID.</li>
+	 *     <li>Other deployment modes can set the resource ID in other ways.</li>
+	 * </ul>
+	 * 
+	 * @return The ID of the resource in which the TaskManager is started
+	 */
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	/**
+	 * Returns the port instance's task manager expects to receive transfer envelopes on.
+	 * 
+	 * @return the port instance's task manager expects to receive transfer envelopes on
+	 */
+	public int dataPort() {
+		return dataPort;
+	}
+
+	/**
+	 * Returns the network address the instance's task manager binds its sockets to.
+	 * 
+	 * @return the network address the instance's task manager binds its sockets to
+	 */
+	public InetAddress address() {
+		return inetAddress;
+	}
+
+	/**
+	 * Gets the IP address where the TaskManager operates.
+	 *
+	 * @return The IP address.
+	 */
+	public String addressString() {
+		return inetAddress.toString();
+	}
+
+	/**
+	 * Returns the fully-qualified domain name the TaskManager. If the name could not be
+	 * determined, the return value will be a textual representation of the TaskManager's IP address.
+	 * 
+	 * @return The fully-qualified domain name of the TaskManager.
+	 */
+	public String getFQDNHostname() {
+		return fqdnHostName;
+	}
+
+	/**
+	 * Gets the hostname of the TaskManager. The hostname derives from the fully qualified
+	 * domain name (FQDN, see {@link #getFQDNHostname()}):
+	 * <ul>
+	 *     <li>If the FQDN is the textual IP address, then the hostname is also the IP address</li>
+	 *     <li>If the FQDN has only one segment (such as "localhost", or "host17"), then this is
+	 *         used as the hostname.</li>
+	 *     <li>If the FQDN has multiple segments (such as "worker3.subgroup.company.net"), then the first
+	 *         segment (here "worker3") will be used as the hostname.</li>
+	 * </ul>
+	 *
+	 * @return The hostname of the TaskManager.
+	 */
+	public String getHostname() {
+		return hostName;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Utilities
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return stringRepresentation;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		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 resourceID.hashCode() + 
+				17 * inetAddress.hashCode() +
+				129 * dataPort;
+	}
+
+	@Override
+	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) {
+			return 1;
+		} else {
+			for (int i = 0; i < thisAddress.length; i++) {
+				byte tb = thisAddress[i];
+				byte ob = otherAddress[i];
+				if (tb < ob) {
+					return -1;
+				} else if (tb > ob) {
+					return 1;
+				}
+			}
+		}
+
+		// addresses are identical, decide based on ports.
+		if (this.dataPort < o.dataPort) {
+			return -1;
+		} else if (this.dataPort > o.dataPort) {
+			return 1;
+		} else {
+			return 0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala
----------------------------------------------------------------------
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 d362164..5648bc6 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
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
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 3154826..84750a3 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.{AkkaActorGateway, HardwareDescription, InstanceID}
 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.buffer.NetworkBufferPool
@@ -127,7 +127,7 @@ import scala.util.{Failure, Success}
 class TaskManager(
     protected val config: TaskManagerConfiguration,
     protected val resourceID: ResourceID,
-    protected val connectionInfo: InstanceConnectionInfo,
+    protected val location: TaskManagerLocation,
     protected val memoryManager: MemoryManager,
     protected val ioManager: IOManager,
     protected val network: NetworkEnvironment,
@@ -189,7 +189,7 @@ class TaskManager(
   var leaderSessionID: Option[UUID] = None
 
   private val runtimeInfo = new TaskManagerRuntimeInfo(
-       connectionInfo.getHostname(),
+       location.getHostname(),
        new UnmodifiableConfiguration(config.configuration),
        config.tmpDirPaths)
 
@@ -209,7 +209,7 @@ class TaskManager(
    */
   override def preStart(): Unit = {
     log.info(s"Starting TaskManager actor at ${self.path.toSerializationFormat}.")
-    log.info(s"TaskManager data connection information: $connectionInfo")
+    log.info(s"TaskManager data connection information: $location")
     log.info(s"TaskManager has $numberOfSlots task slot(s).")
 
     // log the initial memory utilization
@@ -601,7 +601,7 @@ class TaskManager(
             jobManager ! decorateMessage(
               RegisterTaskManager(
                 resourceID,
-                connectionInfo,
+                location,
                 resources,
                 numberOfSlots)
             )
@@ -1884,7 +1884,8 @@ object TaskManager {
 
     network.start()
 
-    val connectionInfo = new InstanceConnectionInfo(
+    val taskManagerLocation = new TaskManagerLocation(
+      resourceID,
       taskManagerAddress.getAddress(),
       network.getConnectionManager().getDataPort())
 
@@ -1994,7 +1995,7 @@ object TaskManager {
       taskManagerClass,
       taskManagerConfig,
       resourceID,
-      connectionInfo,
+      taskManagerLocation,
       memoryManager,
       ioManager,
       network,

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
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 2597753..9b5a147 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(),

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
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 2b8b867..cf7cf58 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.execution.ExecutionState;
 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 class ExecutionGraphMetricsTest extends TestLogger {
 
 		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 class ExecutionGraphMetricsTest extends TestLogger {
 
 		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);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
----------------------------------------------------------------------
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 903d5f9..cddb6cb 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.executiongraph.restart.NoRestartStrategy;
 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 class ExecutionGraphTestUtils {
 	}
 
 	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")

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
----------------------------------------------------------------------
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 2a690d9..a71faf6 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.executiongraph.restart.FixedDelayRestartStrategy
 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 class 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());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java
----------------------------------------------------------------------
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 4ee06b36..91472ae 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.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.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 class VertexLocationConstraintTest {
 	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);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
----------------------------------------------------------------------
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/instance/InstanceConnectionInfoTest.java
deleted file mode 100644
index 3a9488d..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
+++ /dev/null
@@ -1,200 +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.instance;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.net.InetAddress;
-
-import org.apache.flink.util.InstantiationUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the InstanceConnectionInfo, which identifies the location and connection
- * information of a TaskManager.
- */
-public class InstanceConnectionInfoTest {
-
-	@Test
-	public void testEqualsHashAndCompareTo() {
-		try {
-			// we mock the addresses to save the times of the reverse name lookups
-			InetAddress address1 = mock(InetAddress.class);
-			when(address1.getCanonicalHostName()).thenReturn("localhost");
-			when(address1.getHostName()).thenReturn("localhost");
-			when(address1.getHostAddress()).thenReturn("127.0.0.1");
-			when(address1.getAddress()).thenReturn(new byte[] {127, 0, 0, 1} );
-
-			InetAddress address2 = mock(InetAddress.class);
-			when(address2.getCanonicalHostName()).thenReturn("testhost1");
-			when(address2.getHostName()).thenReturn("testhost1");
-			when(address2.getHostAddress()).thenReturn("0.0.0.0");
-			when(address2.getAddress()).thenReturn(new byte[] {0, 0, 0, 0} );
-
-			InetAddress address3 = mock(InetAddress.class);
-			when(address3.getCanonicalHostName()).thenReturn("testhost2");
-			when(address3.getHostName()).thenReturn("testhost2");
-			when(address3.getHostAddress()).thenReturn("192.168.0.1");
-			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);
-			
-			assertTrue(one.equals(four));
-			assertTrue(!one.equals(two));
-			assertTrue(!one.equals(three));
-			assertTrue(!two.equals(three));
-			assertTrue(!three.equals(four));
-			
-			assertTrue(one.compareTo(four) == 0);
-			assertTrue(four.compareTo(one) == 0);
-			assertTrue(one.compareTo(two) != 0);
-			assertTrue(one.compareTo(three) != 0);
-			assertTrue(two.compareTo(three) != 0);
-			assertTrue(three.compareTo(four) != 0);
-			
-			{
-				int val = one.compareTo(two);
-				assertTrue(two.compareTo(one) == -val);
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	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);
-				assertEquals(original, serCopy);
-			}
-						
-			// with resolved hostname
-			{
-				InstanceConnectionInfo original = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871);
-				original.getFQDNHostname();
-				
-				InstanceConnectionInfo copy = InstantiationUtil.createCopyWritable(original);
-				assertEquals(original, copy);
-				
-				InstanceConnectionInfo serCopy = InstantiationUtil.clone(original);
-				assertEquals(original, serCopy);
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testGetFQDNHostname() {
-		try {
-			InstanceConnectionInfo info1 = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871);
-			assertNotNull(info1.getFQDNHostname());
-			
-			InstanceConnectionInfo info2 = new InstanceConnectionInfo(InetAddress.getByName("1.2.3.4"), 8888);
-			assertNotNull(info2.getFQDNHostname());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testGetHostname0() {
-		try {
-			InetAddress address = mock(InetAddress.class);
-			when(address.getCanonicalHostName()).thenReturn("worker2.cluster.mycompany.com");
-			when(address.getHostName()).thenReturn("worker2.cluster.mycompany.com");
-			when(address.getHostAddress()).thenReturn("127.0.0.1");
-
-			final InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871);
-			Assert.assertEquals("worker2", info.getHostname());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testGetHostname1() {
-		try {
-			InetAddress address = mock(InetAddress.class);
-			when(address.getCanonicalHostName()).thenReturn("worker10");
-			when(address.getHostName()).thenReturn("worker10");
-			when(address.getHostAddress()).thenReturn("127.0.0.1");
-
-			InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871);
-			Assert.assertEquals("worker10", info.getHostname());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testGetHostname2() {
-		try {
-			final String addressString = "192.168.254.254";
-
-			// we mock the addresses to save the times of the reverse name lookups
-			InetAddress address = mock(InetAddress.class);
-			when(address.getCanonicalHostName()).thenReturn("192.168.254.254");
-			when(address.getHostName()).thenReturn("192.168.254.254");
-			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);
-
-			assertNotNull(info.getFQDNHostname());
-			assertTrue(info.getFQDNHostname().equals(addressString));
-
-			assertNotNull(info.getHostname());
-			assertTrue(info.getHostname().equals(addressString));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
----------------------------------------------------------------------
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 ff5e2ab..f1ed960 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.Assert;
 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 class InstanceManagerTest{
 			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 class InstanceManagerTest{
 			assertEquals(8, cm.getTotalNumberOfSlots());
 
 			Collection<Instance> instances = cm.getAllRegisteredInstances();
-			Set<InstanceConnectionInfo> instanceConnectionInfos = new
-					HashSet<InstanceConnectionInfo>();
+			Set<TaskManagerLocation> taskManagerLocations = new
+					HashSet<TaskManagerLocation>();
 
 			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 class InstanceManagerTest{
 
 			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 class InstanceManagerTest{
 			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 class InstanceManagerTest{
 			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 class InstanceManagerTest{
 				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,

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
----------------------------------------------------------------------
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 faa679b..82d3723 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 java.net.InetAddress;
 
 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 class InstanceTest {
 	@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 class InstanceTest {
 	@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());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
----------------------------------------------------------------------
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 459a3ed..82c2a74 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.clusterframework.types.ResourceID;
 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 class SimpleSlotTest {
 	}
 
 	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());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
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 983d6e6..99360e2 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 static org.mockito.Mockito.when;
 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.executiongraph.ExecutionVertex;
 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 class SchedulerTestUtils {
 			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 class SchedulerTestUtils {
 		
 		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);
 	}
 	
@@ -143,19 +143,4 @@ public class SchedulerTestUtils {
 		
 		return set.size() == obj.length;
 	}
-	
-	public static boolean areSameSets(Collection<Object> set1, Collection<Object> set2) {
-		if (set1 == null || set2 == null) {
-			throw new IllegalArgumentException();
-		}
-		
-		HashSet<Object> set = new HashSet<Object>(set1);
-		for (Object o : set2) {
-			if (!set.remove(o)) {
-				return false;
-			}
-		}
-		
-		return set.isEmpty();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
----------------------------------------------------------------------
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 ca09634..3307568 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.configuration.Configuration;
 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 @@ public class ResourceManagerITCase extends TestLogger {
 			jobManager.tell(
 				new RegistrationMessages.RegisterTaskManager(
 					resourceID,
-					Mockito.mock(InstanceConnectionInfo.class),
+					Mockito.mock(TaskManagerLocation.class),
 					null,
 					1),
 				me);

http://git-wip-us.apache.org/repos/asf/flink/blob/34cda87a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
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 3371c49..bda4174 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
@@ -25,6 +25,7 @@ import akka.actor.ActorSystem;
 import akka.actor.Kill;
 import akka.actor.Props;
 import akka.testkit.JavaTestKit;
+
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.MemoryType;
@@ -32,7 +33,6 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 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.LocalConnectionManager;
@@ -46,12 +46,12 @@ import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.memory.MemoryManager;
-
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+
 import org.junit.Test;
+
 import scala.Option;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -105,7 +105,9 @@ public class TaskManagerComponentsStartupShutdownTest {
 					32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0,
 					Option.<NettyConfig>empty(), 0, 0);
 
-			final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000);
+			ResourceID taskManagerId = ResourceID.generate();
+			
+			final TaskManagerLocation connectionInfo = new TaskManagerLocation(taskManagerId, InetAddress.getLocalHost(), 10000);
 
 			final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false);
 			final IOManager ioManager = new IOManagerAsync(TMP_DIR);
@@ -130,7 +132,7 @@ public class TaskManagerComponentsStartupShutdownTest {
 			final Props tmProps = Props.create(
 				TaskManager.class,
 				tmConfig,
-				ResourceID.generate(),
+				taskManagerId,
 				connectionInfo,
 				memManager,
 				ioManager,


[5/8] flink git commit: [FLINK-4566] [network runtime] Properly preserve exception causes for ProducerFailedException

Posted by se...@apache.org.
[FLINK-4566] [network runtime] Properly preserve exception causes for ProducerFailedException


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/e227b101
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e227b101
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e227b101

Branch: refs/heads/master
Commit: e227b10134e387f3c49804dc0cc4c223c30702e3
Parents: 761d0a0
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 2 11:45:25 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../partition/ProducerFailedException.java       | 19 +++++--------------
 .../partition/ProducerFailedExceptionTest.java   | 12 ++++++------
 2 files changed, 11 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e227b101/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
index 2b2acab..934234d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.runtime.execution.CancelTaskException;
-import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.runtime.util.SerializedThrowable;
 
 /**
  * Network-stack level Exception to notify remote receiver about a failed
@@ -29,23 +29,14 @@ public class ProducerFailedException extends CancelTaskException {
 
 	private static final long serialVersionUID = -1555492656299526395L;
 
-	private final String causeAsString;
-
 	/**
 	 * The cause of the producer failure.
 	 *
-	 * Note: The cause will be stringified, because it might be an instance of
-	 * a user level Exception, which can not be deserialized by the remote
-	 * receiver's system class loader.
+	 * <p>The cause will be stored as a {@link SerializedThrowable}, because it might
+	 * be an instance of a user level Exception, which may not be possible to deserialize
+	 * by the remote receiver's system class loader.
 	 */
 	public ProducerFailedException(Throwable cause) {
-		this.causeAsString = cause != null ? ExceptionUtils.stringifyException(cause) : null;
-	}
-
-	/**
-	 * Returns the stringified cause of the producer failure.
-	 */
-	public String getCauseAsString() {
-		return causeAsString;
+		super(new SerializedThrowable(cause));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e227b101/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
index 042c136..ca2de0c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
@@ -19,27 +19,27 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.runtime.execution.CancelTaskException;
+import org.apache.flink.runtime.util.SerializedThrowable;
+
 import org.junit.Test;
 
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 public class ProducerFailedExceptionTest {
 
 	@Test
 	public void testInstanceOfCancelTaskException() throws Exception {
-		ProducerFailedException e = new ProducerFailedException(new Exception());
-		assertTrue(e instanceof CancelTaskException);
+		assertTrue(CancelTaskException.class.isAssignableFrom(ProducerFailedException.class));
 	}
 
 	@Test
-	public void testCauseIsStringified() throws Exception {
+	public void testCauseIsSerialized() throws Exception {
 		// Tests that the cause is stringified, because it might be an instance
 		// of a user level Exception, which can not be deserialized by the
 		// remote receiver's system class loader.
 		ProducerFailedException e = new ProducerFailedException(new Exception());
-		assertNull(e.getCause());
-		assertNotNull(e.getCauseAsString());
+		assertNotNull(e.getCause());
+		assertTrue(e.getCause() instanceof SerializedThrowable);
 	}
 }