You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2017/09/19 22:44:11 UTC

[01/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Repository: flink
Updated Branches:
  refs/heads/master 327701032 -> 4fc019a96


http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionConfigBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionConfigBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionConfigBuilder.java
new file mode 100644
index 0000000..cb7fe90
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionConfigBuilder.java
@@ -0,0 +1,71 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.api.common.ArchivedExecutionConfig;
+import org.apache.flink.api.common.ExecutionMode;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Utility class for constructing an ArchivedExecutionConfig.
+ */
+public class ArchivedExecutionConfigBuilder {
+	private String executionMode;
+	private String restartStrategyDescription;
+	private int parallelism;
+	private boolean objectReuseEnabled;
+	private Map<String, String> globalJobParameters;
+
+	public ArchivedExecutionConfigBuilder setExecutionMode(String executionMode) {
+		this.executionMode = executionMode;
+		return this;
+	}
+
+	public ArchivedExecutionConfigBuilder setRestartStrategyDescription(String restartStrategyDescription) {
+		this.restartStrategyDescription = restartStrategyDescription;
+		return this;
+	}
+
+	public ArchivedExecutionConfigBuilder setParallelism(int parallelism) {
+		this.parallelism = parallelism;
+		return this;
+	}
+
+	public ArchivedExecutionConfigBuilder setObjectReuseEnabled(boolean objectReuseEnabled) {
+		this.objectReuseEnabled = objectReuseEnabled;
+		return this;
+	}
+
+	public ArchivedExecutionConfigBuilder setGlobalJobParameters(Map<String, String> globalJobParameters) {
+		this.globalJobParameters = globalJobParameters;
+		return this;
+	}
+
+	public ArchivedExecutionConfig build() {
+		return new ArchivedExecutionConfig(
+			executionMode != null ? executionMode : ExecutionMode.PIPELINED.name(),
+			restartStrategyDescription != null ? restartStrategyDescription : "default",
+			parallelism,
+			objectReuseEnabled,
+			globalJobParameters != null ? globalJobParameters : Collections.<String, String>emptyMap()
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java
new file mode 100644
index 0000000..68077ba
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java
@@ -0,0 +1,140 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.api.common.ArchivedExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * Utility class for constructing an ArchivedExecutionGraph.
+ */
+public class ArchivedExecutionGraphBuilder {
+
+	private static final Random RANDOM = new Random();
+
+	private JobID jobID;
+	private String jobName;
+	private Map<JobVertexID, ArchivedExecutionJobVertex> tasks;
+	private List<ArchivedExecutionJobVertex> verticesInCreationOrder;
+	private long[] stateTimestamps;
+	private JobStatus state;
+	private ErrorInfo failureCause;
+	private String jsonPlan;
+	private StringifiedAccumulatorResult[] archivedUserAccumulators;
+	private ArchivedExecutionConfig archivedExecutionConfig;
+	private boolean isStoppable;
+	private Map<String, SerializedValue<Object>> serializedUserAccumulators;
+
+	public ArchivedExecutionGraphBuilder setJobID(JobID jobID) {
+		this.jobID = jobID;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setJobName(String jobName) {
+		this.jobName = jobName;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setTasks(Map<JobVertexID, ArchivedExecutionJobVertex> tasks) {
+		this.tasks = tasks;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setVerticesInCreationOrder(List<ArchivedExecutionJobVertex> verticesInCreationOrder) {
+		this.verticesInCreationOrder = verticesInCreationOrder;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setStateTimestamps(long[] stateTimestamps) {
+		Preconditions.checkArgument(stateTimestamps.length == JobStatus.values().length);
+		this.stateTimestamps = stateTimestamps;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setState(JobStatus state) {
+		this.state = state;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setFailureCause(ErrorInfo failureCause) {
+		this.failureCause = failureCause;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setJsonPlan(String jsonPlan) {
+		this.jsonPlan = jsonPlan;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setArchivedUserAccumulators(StringifiedAccumulatorResult[] archivedUserAccumulators) {
+		this.archivedUserAccumulators = archivedUserAccumulators;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setArchivedExecutionConfig(ArchivedExecutionConfig archivedExecutionConfig) {
+		this.archivedExecutionConfig = archivedExecutionConfig;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setStoppable(boolean stoppable) {
+		isStoppable = stoppable;
+		return this;
+	}
+
+	public ArchivedExecutionGraphBuilder setSerializedUserAccumulators(Map<String, SerializedValue<Object>> serializedUserAccumulators) {
+		this.serializedUserAccumulators = serializedUserAccumulators;
+		return this;
+	}
+
+	public ArchivedExecutionGraph build() {
+		Preconditions.checkNotNull(tasks, "Tasks must not be null.");
+		JobID jobID = this.jobID != null ? this.jobID : new JobID();
+		String jobName = this.jobName != null ? this.jobName : "job_" + RANDOM.nextInt();
+		return new ArchivedExecutionGraph(
+			jobID,
+			jobName,
+			tasks,
+			verticesInCreationOrder != null ? verticesInCreationOrder : new ArrayList<>(tasks.values()),
+			stateTimestamps != null ? stateTimestamps : new long[JobStatus.values().length],
+			state != null ? state : JobStatus.FINISHED,
+			failureCause,
+			jsonPlan != null ? jsonPlan : "{\"jobid\":\"" + jobID + "\", \"name\":\"" + jobName + "\", \"nodes\":[]}",
+			archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0],
+			serializedUserAccumulators != null ? serializedUserAccumulators : Collections.<String, SerializedValue<Object>>emptyMap(),
+			archivedExecutionConfig != null ? archivedExecutionConfig : new ArchivedExecutionConfigBuilder().build(),
+			isStoppable,
+			null,
+			null
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionJobVertexBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionJobVertexBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionJobVertexBuilder.java
new file mode 100644
index 0000000..814c4db
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionJobVertexBuilder.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Random;
+
+/**
+ * Utility class for constructing an ArchivedExecutionJobVertex.
+ */
+public class ArchivedExecutionJobVertexBuilder {
+
+	private static final Random RANDOM = new Random();
+
+	private ArchivedExecutionVertex[] taskVertices;
+	private JobVertexID id;
+	private String name;
+	private int parallelism;
+	private int maxParallelism;
+	private StringifiedAccumulatorResult[] archivedUserAccumulators;
+
+	public ArchivedExecutionJobVertexBuilder setTaskVertices(ArchivedExecutionVertex[] taskVertices) {
+		this.taskVertices = taskVertices;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertexBuilder setId(JobVertexID id) {
+		this.id = id;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertexBuilder setName(String name) {
+		this.name = name;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertexBuilder setParallelism(int parallelism) {
+		this.parallelism = parallelism;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertexBuilder setMaxParallelism(int maxParallelism) {
+		this.maxParallelism = maxParallelism;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertexBuilder setArchivedUserAccumulators(StringifiedAccumulatorResult[] archivedUserAccumulators) {
+		this.archivedUserAccumulators = archivedUserAccumulators;
+		return this;
+	}
+
+	public ArchivedExecutionJobVertex build() {
+		Preconditions.checkNotNull(taskVertices);
+		return new ArchivedExecutionJobVertex(
+			taskVertices,
+			id != null ? id : new JobVertexID(),
+			name != null ? name : "task_" + RANDOM.nextInt(),
+			parallelism,
+			maxParallelism,
+			archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0]
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionVertexBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionVertexBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionVertexBuilder.java
new file mode 100644
index 0000000..935c792
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionVertexBuilder.java
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.runtime.executiongraph.ArchivedExecution;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.util.EvictingBoundedList;
+import org.apache.flink.util.Preconditions;
+
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Utility class for constructing an ArchivedExecutionVertex.
+ */
+public class ArchivedExecutionVertexBuilder {
+
+	private static final Random RANDOM = new Random();
+
+	private int subtaskIndex;
+	private EvictingBoundedList<ArchivedExecution> priorExecutions;
+	private String taskNameWithSubtask;
+	private ArchivedExecution currentExecution;
+
+	public ArchivedExecutionVertexBuilder setSubtaskIndex(int subtaskIndex) {
+		this.subtaskIndex = subtaskIndex;
+		return this;
+	}
+
+	public ArchivedExecutionVertexBuilder setPriorExecutions(List<ArchivedExecution> priorExecutions) {
+		this.priorExecutions = new EvictingBoundedList<>(priorExecutions.size());
+		for (ArchivedExecution execution : priorExecutions) {
+			this.priorExecutions.add(execution);
+		}
+		return this;
+	}
+
+	public ArchivedExecutionVertexBuilder setTaskNameWithSubtask(String taskNameWithSubtask) {
+		this.taskNameWithSubtask = taskNameWithSubtask;
+		return this;
+	}
+
+	public ArchivedExecutionVertexBuilder setCurrentExecution(ArchivedExecution currentExecution) {
+		this.currentExecution = currentExecution;
+		return this;
+	}
+
+	public ArchivedExecutionVertex build() {
+		Preconditions.checkNotNull(currentExecution);
+		return new ArchivedExecutionVertex(
+			subtaskIndex,
+			taskNameWithSubtask != null ? taskNameWithSubtask : "task_" + RANDOM.nextInt() + "_" + subtaskIndex,
+			currentExecution,
+			priorExecutions != null ? priorExecutions : new EvictingBoundedList<ArchivedExecution>(0)
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedJobGenerationUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedJobGenerationUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedJobGenerationUtils.java
new file mode 100644
index 0000000..d256e92
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedJobGenerationUtils.java
@@ -0,0 +1,164 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecution;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Common entry-point for accessing generated ArchivedExecution* components.
+ */
+public class ArchivedJobGenerationUtils {
+	public static final ObjectMapper MAPPER = new ObjectMapper();
+	public static final JsonFactory JACKSON_FACTORY = new JsonFactory()
+		.enable(JsonGenerator.Feature.AUTO_CLOSE_TARGET)
+		.disable(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT);
+
+	private static ArchivedExecutionGraph originalJob;
+	private static ArchivedExecutionJobVertex originalTask;
+	private static ArchivedExecutionVertex originalSubtask;
+	private static ArchivedExecution originalAttempt;
+
+	private static final Object lock = new Object();
+
+	private ArchivedJobGenerationUtils() {
+	}
+
+	public static AccessExecutionGraph getTestJob() throws Exception {
+		synchronized (lock) {
+			if (originalJob == null) {
+				generateArchivedJob();
+			}
+		}
+		return originalJob;
+	}
+
+	public static AccessExecutionJobVertex getTestTask() throws Exception {
+		synchronized (lock) {
+			if (originalJob == null) {
+				generateArchivedJob();
+			}
+		}
+		return originalTask;
+	}
+
+	public static AccessExecutionVertex getTestSubtask() throws Exception {
+		synchronized (lock) {
+			if (originalJob == null) {
+				generateArchivedJob();
+			}
+		}
+		return originalSubtask;
+	}
+
+	public static AccessExecution getTestAttempt() throws Exception {
+		synchronized (lock) {
+			if (originalJob == null) {
+				generateArchivedJob();
+			}
+		}
+		return originalAttempt;
+	}
+
+	private static void generateArchivedJob() throws Exception {
+		// Attempt
+		StringifiedAccumulatorResult acc1 = new StringifiedAccumulatorResult("name1", "type1", "value1");
+		StringifiedAccumulatorResult acc2 = new StringifiedAccumulatorResult("name2", "type2", "value2");
+		TaskManagerLocation location = new TaskManagerLocation(new ResourceID("hello"), InetAddress.getLocalHost(), 1234);
+		originalAttempt = new ArchivedExecutionBuilder()
+			.setStateTimestamps(new long[]{1, 2, 3, 4, 5, 6, 7, 8, 9})
+			.setParallelSubtaskIndex(1)
+			.setAttemptNumber(0)
+			.setAssignedResourceLocation(location)
+			.setUserAccumulators(new StringifiedAccumulatorResult[]{acc1, acc2})
+			.setState(ExecutionState.FINISHED)
+			.setFailureCause("attemptException")
+			.build();
+		// Subtask
+		originalSubtask = new ArchivedExecutionVertexBuilder()
+			.setSubtaskIndex(originalAttempt.getParallelSubtaskIndex())
+			.setTaskNameWithSubtask("hello(1/1)")
+			.setCurrentExecution(originalAttempt)
+			.build();
+		// Task
+		originalTask = new ArchivedExecutionJobVertexBuilder()
+			.setTaskVertices(new ArchivedExecutionVertex[]{originalSubtask})
+			.build();
+		// Job
+		Map<JobVertexID, ArchivedExecutionJobVertex> tasks = new HashMap<>();
+		tasks.put(originalTask.getJobVertexId(), originalTask);
+		originalJob = new ArchivedExecutionGraphBuilder()
+			.setJobID(new JobID())
+			.setTasks(tasks)
+			.setFailureCause(new ErrorInfo(new Exception("jobException"), originalAttempt.getStateTimestamp(ExecutionState.FAILED)))
+			.setState(JobStatus.FINISHED)
+			.setStateTimestamps(new long[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
+			.setArchivedUserAccumulators(new StringifiedAccumulatorResult[]{acc1, acc2})
+			.build();
+	}
+
+	// ========================================================================
+	// utility methods
+	// ========================================================================
+
+	public static void compareStringifiedAccumulators(StringifiedAccumulatorResult[] expectedAccs, ArrayNode writtenAccs) {
+		assertEquals(expectedAccs.length, writtenAccs.size());
+		for (int x = 0; x < expectedAccs.length; x++) {
+			JsonNode acc = writtenAccs.get(x);
+
+			assertEquals(expectedAccs[x].getName(), acc.get("name").asText());
+			assertEquals(expectedAccs[x].getType(), acc.get("type").asText());
+			assertEquals(expectedAccs[x].getValue(), acc.get("value").asText());
+		}
+	}
+
+	public static void compareIoMetrics(IOMetrics expectedMetrics, JsonNode writtenMetrics) {
+		assertEquals(expectedMetrics.getNumBytesInTotal(), writtenMetrics.get("read-bytes").asLong());
+		assertEquals(expectedMetrics.getNumBytesOut(), writtenMetrics.get("write-bytes").asLong());
+		assertEquals(expectedMetrics.getNumRecordsIn(), writtenMetrics.get("read-records").asLong());
+		assertEquals(expectedMetrics.getNumRecordsOut(), writtenMetrics.get("write-records").asLong());
+	}
+}


[08/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/WebHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/WebHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/WebHandler.java
new file mode 100644
index 0000000..58fda14
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/WebHandler.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rest.handler;
+
+/**
+ * Marker interface for web handlers which can describe their paths.
+ */
+public interface WebHandler {
+
+	/**
+	 * Returns an array of REST URL's under which this handler can be registered.
+	 *
+	 * @return array containing REST URL's under which this handler can be registered.
+	 */
+	String[] getPaths();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractExecutionGraphRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractExecutionGraphRequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractExecutionGraphRequestHandler.java
new file mode 100644
index 0000000..e214a36
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractExecutionGraphRequestHandler.java
@@ -0,0 +1,79 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.NotFoundException;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for request handlers whose response depends on an ExecutionGraph
+ * that can be retrieved via "jobid" parameter.
+ */
+public abstract class AbstractExecutionGraphRequestHandler extends AbstractJsonRequestHandler {
+
+	private final ExecutionGraphHolder executionGraphHolder;
+
+	public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executor);
+		this.executionGraphHolder = Preconditions.checkNotNull(executionGraphHolder);
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(
+			Map<String, String> pathParams,
+			Map<String, String> queryParams,
+			JobManagerGateway jobManagerGateway) {
+		String jidString = pathParams.get("jobid");
+		if (jidString == null) {
+			throw new RuntimeException("JobId parameter missing");
+		}
+
+		JobID jid;
+		try {
+			jid = JobID.fromHexString(jidString);
+		}
+		catch (Exception e) {
+			return FutureUtils.completedExceptionally(new FlinkException("Invalid JobID string '" + jidString + "'", e));
+		}
+
+		final CompletableFuture<Optional<AccessExecutionGraph>> graphFuture = executionGraphHolder.getExecutionGraph(jid, jobManagerGateway);
+
+		return graphFuture.thenComposeAsync(
+			(Optional<AccessExecutionGraph> optGraph) -> {
+				if (optGraph.isPresent()) {
+					return handleRequest(optGraph.get(), pathParams);
+				} else {
+					throw new FlinkFutureException(new NotFoundException("Could not find job with jobId " + jid + '.'));
+				}
+			}, executor);
+	}
+
+	public abstract CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJobVertexRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJobVertexRequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJobVertexRequestHandler.java
new file mode 100644
index 0000000..e2e4484
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJobVertexRequestHandler.java
@@ -0,0 +1,71 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for request handlers whose response depends on a specific job vertex (defined
+ * via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
+ */
+public abstract class AbstractJobVertexRequestHandler extends AbstractExecutionGraphRequestHandler {
+
+	public AbstractJobVertexRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public final CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		final JobVertexID vid = parseJobVertexId(params);
+
+		final AccessExecutionJobVertex jobVertex = graph.getJobVertex(vid);
+		if (jobVertex == null) {
+			throw new IllegalArgumentException("No vertex with ID '" + vid + "' exists.");
+		}
+
+		return handleRequest(jobVertex, params);
+	}
+
+	/**
+	 * Returns the job vertex ID parsed from the provided parameters.
+	 *
+	 * @param params Path parameters
+	 * @return Parsed job vertex ID or <code>null</code> if not available.
+	 */
+	public static JobVertexID parseJobVertexId(Map<String, String> params) {
+		String jobVertexIdParam = params.get("vertexid");
+		if (jobVertexIdParam == null) {
+			return null;
+		}
+
+		try {
+			return JobVertexID.fromHexString(jobVertexIdParam);
+		} catch (RuntimeException ignored) {
+			return null;
+		}
+	}
+
+	public abstract CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJsonRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJsonRequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJsonRequestHandler.java
new file mode 100644
index 0000000..43c4af3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractJsonRequestHandler.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.NotFoundException;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+
+import java.nio.charset.Charset;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for most request handlers. The handlers must produce a JSON response.
+ */
+public abstract class AbstractJsonRequestHandler implements RequestHandler {
+
+	private static final Charset ENCODING = Charset.forName("UTF-8");
+
+	protected final Executor executor;
+
+	protected AbstractJsonRequestHandler(Executor executor) {
+		this.executor = Preconditions.checkNotNull(executor);
+	}
+
+	@Override
+	public CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		CompletableFuture<String> resultFuture = handleJsonRequest(pathParams, queryParams, jobManagerGateway);
+
+		return resultFuture.thenApplyAsync(
+			(String result) -> {
+				byte[] bytes = result.getBytes(ENCODING);
+
+				DefaultFullHttpResponse response = new DefaultFullHttpResponse(
+					HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes));
+
+				response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
+				response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
+
+				return response;
+			});
+	}
+
+	/**
+	 * Core method that handles the request and generates the response. The method needs to
+	 * respond with a valid JSON string. Exceptions may be thrown and will be handled.
+	 *
+	 * @param pathParams The map of REST path parameters, decoded by the router.
+	 * @param queryParams The map of query parameters.
+	 * @param jobManagerGateway to communicate with the JobManager.
+	 *
+	 * @return The JSON string that is the HTTP response.
+	 *
+	 * @throws Exception Handlers may forward exceptions. Exceptions of type
+	 *         {@link NotFoundException} will cause a HTTP 404
+	 *         response with the exception message, other exceptions will cause a HTTP 500 response
+	 *         with the exception stack trace.
+	 */
+	public abstract CompletableFuture<String> handleJsonRequest(
+			Map<String, String> pathParams,
+			Map<String, String> queryParams,
+			JobManagerGateway jobManagerGateway);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskAttemptRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskAttemptRequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskAttemptRequestHandler.java
new file mode 100644
index 0000000..ec277d8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskAttemptRequestHandler.java
@@ -0,0 +1,77 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.util.FlinkException;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for request handlers whose response depends on a specific subtask execution attempt
+ * (defined via the "attempt" parameter) of a specific subtask (defined via the
+ * "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
+ * specific job, defined via (defined voa the "jobid" parameter).
+ */
+public abstract class AbstractSubtaskAttemptRequestHandler extends AbstractSubtaskRequestHandler {
+
+	public AbstractSubtaskAttemptRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params) {
+		final String attemptNumberString = params.get("attempt");
+		if (attemptNumberString == null) {
+			return FutureUtils.completedExceptionally(new FlinkException("Attempt number parameter missing"));
+		}
+
+		final int attempt;
+		try {
+			attempt = Integer.parseInt(attemptNumberString);
+		}
+		catch (NumberFormatException e) {
+			return FutureUtils.completedExceptionally(new FlinkException("Invalid attempt number parameter"));
+		}
+
+		final AccessExecution currentAttempt = vertex.getCurrentExecutionAttempt();
+		if (attempt == currentAttempt.getAttemptNumber()) {
+			return handleRequest(currentAttempt, params);
+		}
+		else if (attempt >= 0 && attempt < currentAttempt.getAttemptNumber()) {
+			AccessExecution exec = vertex.getPriorExecutionAttempt(attempt);
+
+			if (exec != null) {
+				return handleRequest(exec, params);
+			} else {
+				return FutureUtils.completedExceptionally(new RequestHandlerException("Execution for attempt " + attempt +
+					" has already been deleted."));
+			}
+		}
+		else {
+			return FutureUtils.completedExceptionally(new FlinkException("Attempt does not exist: " + attempt));
+		}
+	}
+
+	public abstract CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskRequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskRequestHandler.java
new file mode 100644
index 0000000..d69038a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/AbstractSubtaskRequestHandler.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.util.FlinkException;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for request handlers whose response depends on a specific subtask (defined via the
+ * "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
+ * specific job, defined via (defined voa the "jobid" parameter).
+ */
+public abstract class AbstractSubtaskRequestHandler extends AbstractJobVertexRequestHandler {
+
+	public AbstractSubtaskRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public final CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		final String subtaskNumberString = params.get("subtasknum");
+		if (subtaskNumberString == null) {
+			return FutureUtils.completedExceptionally(new FlinkException("Subtask number parameter missing"));
+		}
+
+		final int subtask;
+		try {
+			subtask = Integer.parseInt(subtaskNumberString);
+		}
+		catch (NumberFormatException e) {
+			return FutureUtils.completedExceptionally(new FlinkException("Invalid subtask number parameter", e));
+		}
+
+		if (subtask < 0 || subtask >= jobVertex.getParallelism()) {
+			return FutureUtils.completedExceptionally(new FlinkException("subtask does not exist: " + subtask));
+		}
+
+		final AccessExecutionVertex vertex = jobVertex.getTaskVertices()[subtask];
+		return handleRequest(vertex, params);
+	}
+
+	public abstract CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
new file mode 100644
index 0000000..db13633
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
@@ -0,0 +1,105 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.webmonitor.StatusOverview;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.util.FlinkException;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Responder that returns the status of the Flink cluster, such as how many
+ * TaskManagers are currently connected, and how many jobs are running.
+ */
+public class ClusterOverviewHandler extends AbstractJsonRequestHandler {
+
+	private static final String CLUSTER_OVERVIEW_REST_PATH = "/overview";
+
+	private static final String version = EnvironmentInformation.getVersion();
+
+	private static final String commitID = EnvironmentInformation.getRevisionInformation().commitId;
+
+	private final Time timeout;
+
+	public ClusterOverviewHandler(Executor executor, Time timeout) {
+		super(executor);
+		this.timeout = checkNotNull(timeout);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CLUSTER_OVERVIEW_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		// we need no parameters, get all requests
+		try {
+			if (jobManagerGateway != null) {
+				CompletableFuture<StatusOverview> overviewFuture = jobManagerGateway.requestStatusOverview(timeout);
+
+				return overviewFuture.thenApplyAsync(
+					(StatusOverview overview) -> {
+						StringWriter writer = new StringWriter();
+						try {
+							JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+							gen.writeStartObject();
+							gen.writeNumberField("taskmanagers", overview.getNumTaskManagersConnected());
+							gen.writeNumberField("slots-total", overview.getNumSlotsTotal());
+							gen.writeNumberField("slots-available", overview.getNumSlotsAvailable());
+							gen.writeNumberField("jobs-running", overview.getNumJobsRunningOrPending());
+							gen.writeNumberField("jobs-finished", overview.getNumJobsFinished());
+							gen.writeNumberField("jobs-cancelled", overview.getNumJobsCancelled());
+							gen.writeNumberField("jobs-failed", overview.getNumJobsFailed());
+							gen.writeStringField("flink-version", version);
+							if (!commitID.equals(EnvironmentInformation.UNKNOWN)) {
+								gen.writeStringField("flink-commit", commitID);
+							}
+							gen.writeEndObject();
+
+							gen.close();
+							return writer.toString();
+						} catch (IOException exception) {
+							throw new FlinkFutureException("Could not write cluster overview.", exception);
+						}
+					},
+					executor);
+			} else {
+				throw new Exception("No connection to the leading JobManager.");
+			}
+		}
+		catch (Exception e) {
+			return FutureUtils.completedExceptionally(new FlinkException("Failed to fetch list of all running jobs: ", e));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java
new file mode 100644
index 0000000..57214f0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.configuration.ConfigConstants;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+
+/**
+ * Responder that returns a constant String.
+ */
+@ChannelHandler.Sharable
+public class ConstantTextHandler extends SimpleChannelInboundHandler<Routed> {
+
+	private final byte[] encodedText;
+
+	public ConstantTextHandler(String text) {
+		this.encodedText = text.getBytes(ConfigConstants.DEFAULT_CHARSET);
+	}
+
+	@Override
+	protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
+		HttpResponse response = new DefaultFullHttpResponse(
+			HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(encodedText));
+
+		response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, encodedText.length);
+		response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain");
+
+		KeepAliveWrite.flush(ctx, routed.request(), response);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java
new file mode 100644
index 0000000..07d9707
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java
@@ -0,0 +1,112 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Responder that returns with a list of all JobIDs of jobs found at the target actor.
+ * May serve the IDs of current jobs, or past jobs, depending on whether this handler is
+ * given the JobManager or Archive Actor Reference.
+ */
+public class CurrentJobIdsHandler extends AbstractJsonRequestHandler {
+
+	private static final String CURRENT_JOB_IDS_REST_PATH = "/jobs";
+
+	private final Time timeout;
+
+	public CurrentJobIdsHandler(Executor executor, Time timeout) {
+		super(executor);
+		this.timeout = requireNonNull(timeout);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CURRENT_JOB_IDS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				// we need no parameters, get all requests
+				try {
+					if (jobManagerGateway != null) {
+						CompletableFuture<JobsWithIDsOverview> overviewFuture = jobManagerGateway.requestJobsOverview(timeout);
+						JobsWithIDsOverview overview = overviewFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS);
+
+						StringWriter writer = new StringWriter();
+						JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+						gen.writeStartObject();
+
+						gen.writeArrayFieldStart("jobs-running");
+						for (JobID jid : overview.getJobsRunningOrPending()) {
+							gen.writeString(jid.toString());
+						}
+						gen.writeEndArray();
+
+						gen.writeArrayFieldStart("jobs-finished");
+						for (JobID jid : overview.getJobsFinished()) {
+							gen.writeString(jid.toString());
+						}
+						gen.writeEndArray();
+
+						gen.writeArrayFieldStart("jobs-cancelled");
+						for (JobID jid : overview.getJobsCancelled()) {
+							gen.writeString(jid.toString());
+						}
+						gen.writeEndArray();
+
+						gen.writeArrayFieldStart("jobs-failed");
+						for (JobID jid : overview.getJobsFailed()) {
+							gen.writeString(jid.toString());
+						}
+						gen.writeEndArray();
+
+						gen.writeEndObject();
+
+						gen.close();
+						return writer.toString();
+					}
+					else {
+						throw new Exception("No connection to the leading JobManager.");
+					}
+				}
+				catch (Exception e) {
+					throw new FlinkFutureException("Failed to fetch list of all running jobs.", e);
+				}
+			},
+			executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandler.java
new file mode 100644
index 0000000..6f85320
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandler.java
@@ -0,0 +1,182 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
+import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Request handler that returns a summary of the job status.
+ */
+public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
+
+	private static final String ALL_JOBS_REST_PATH = "/joboverview";
+	private static final String RUNNING_JOBS_REST_PATH = "/joboverview/running";
+	private static final String COMPLETED_JOBS_REST_PATH = "/joboverview/completed";
+
+	private final Time timeout;
+
+	private final boolean includeRunningJobs;
+	private final boolean includeFinishedJobs;
+
+	public CurrentJobsOverviewHandler(
+			Executor executor,
+			Time timeout,
+			boolean includeRunningJobs,
+			boolean includeFinishedJobs) {
+
+		super(executor);
+		this.timeout = checkNotNull(timeout);
+		this.includeRunningJobs = includeRunningJobs;
+		this.includeFinishedJobs = includeFinishedJobs;
+	}
+
+	@Override
+	public String[] getPaths() {
+		if (includeRunningJobs && includeFinishedJobs) {
+			return new String[]{ALL_JOBS_REST_PATH};
+		}
+		if (includeRunningJobs) {
+			return new String[]{RUNNING_JOBS_REST_PATH};
+		} else {
+			return new String[]{COMPLETED_JOBS_REST_PATH};
+		}
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		if (jobManagerGateway != null) {
+			CompletableFuture<MultipleJobsDetails> jobDetailsFuture = jobManagerGateway.requestJobDetails(includeRunningJobs, includeFinishedJobs, timeout);
+
+			return jobDetailsFuture.thenApplyAsync(
+				(MultipleJobsDetails result) -> {
+					final long now = System.currentTimeMillis();
+
+					StringWriter writer = new StringWriter();
+					try {
+						JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+						gen.writeStartObject();
+
+						if (includeRunningJobs && includeFinishedJobs) {
+							gen.writeArrayFieldStart("running");
+							for (JobDetails detail : result.getRunningJobs()) {
+								writeJobDetailOverviewAsJson(detail, gen, now);
+							}
+							gen.writeEndArray();
+
+							gen.writeArrayFieldStart("finished");
+							for (JobDetails detail : result.getFinishedJobs()) {
+								writeJobDetailOverviewAsJson(detail, gen, now);
+							}
+							gen.writeEndArray();
+						} else {
+							gen.writeArrayFieldStart("jobs");
+							for (JobDetails detail : includeRunningJobs ? result.getRunningJobs() : result.getFinishedJobs()) {
+								writeJobDetailOverviewAsJson(detail, gen, now);
+							}
+							gen.writeEndArray();
+						}
+
+						gen.writeEndObject();
+						gen.close();
+						return writer.toString();
+					} catch (IOException e) {
+						throw new FlinkFutureException("Could not write current jobs overview json.", e);
+					}
+				},
+				executor);
+		}
+		else {
+			return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
+		}
+	}
+
+	/**
+	 * Archivist for the CurrentJobsOverviewHandler.
+	 */
+	public static class CurrentJobsOverviewJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			StringWriter writer = new StringWriter();
+			try (JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
+				gen.writeStartObject();
+				gen.writeArrayFieldStart("running");
+				gen.writeEndArray();
+				gen.writeArrayFieldStart("finished");
+				writeJobDetailOverviewAsJson(WebMonitorUtils.createDetailsForJob(graph), gen, System.currentTimeMillis());
+				gen.writeEndArray();
+				gen.writeEndObject();
+			}
+			String json = writer.toString();
+			String path = ALL_JOBS_REST_PATH;
+			return Collections.singleton(new ArchivedJson(path, json));
+		}
+	}
+
+	public static void writeJobDetailOverviewAsJson(JobDetails details, JsonGenerator gen, long now) throws IOException {
+		gen.writeStartObject();
+
+		gen.writeStringField("jid", details.getJobId().toString());
+		gen.writeStringField("name", details.getJobName());
+		gen.writeStringField("state", details.getStatus().name());
+
+		gen.writeNumberField("start-time", details.getStartTime());
+		gen.writeNumberField("end-time", details.getEndTime());
+		gen.writeNumberField("duration", (details.getEndTime() <= 0 ? now : details.getEndTime()) - details.getStartTime());
+		gen.writeNumberField("last-modification", details.getLastUpdateTime());
+
+		gen.writeObjectFieldStart("tasks");
+		gen.writeNumberField("total", details.getNumTasks());
+
+		final int[] perState = details.getNumVerticesPerExecutionState();
+		gen.writeNumberField("pending", perState[ExecutionState.CREATED.ordinal()] +
+				perState[ExecutionState.SCHEDULED.ordinal()] +
+				perState[ExecutionState.DEPLOYING.ordinal()]);
+		gen.writeNumberField("running", perState[ExecutionState.RUNNING.ordinal()]);
+		gen.writeNumberField("finished", perState[ExecutionState.FINISHED.ordinal()]);
+		gen.writeNumberField("canceling", perState[ExecutionState.CANCELING.ordinal()]);
+		gen.writeNumberField("canceled", perState[ExecutionState.CANCELED.ordinal()]);
+		gen.writeNumberField("failed", perState[ExecutionState.FAILED.ordinal()]);
+		gen.writeEndObject();
+
+		gen.writeEndObject();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
new file mode 100644
index 0000000..e8854f4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
@@ -0,0 +1,90 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Responder that returns the parameters that define how the asynchronous requests
+ * against this web server should behave. It defines for example the refresh interval,
+ * and time zone of the server timestamps.
+ */
+public class DashboardConfigHandler extends AbstractJsonRequestHandler {
+
+	private static final String DASHBOARD_CONFIG_REST_PATH = "/config";
+
+	private final String configString;
+
+	public DashboardConfigHandler(Executor executor, long refreshInterval) {
+		super(executor);
+		try {
+			this.configString = createConfigJson(refreshInterval);
+		}
+		catch (Exception e) {
+			// should never happen
+			throw new RuntimeException(e.getMessage(), e);
+		}
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{DASHBOARD_CONFIG_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.completedFuture(configString);
+	}
+
+	public static String createConfigJson(long refreshInterval) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		TimeZone timeZone = TimeZone.getDefault();
+		String timeZoneName = timeZone.getDisplayName();
+		long timeZoneOffset = timeZone.getRawOffset();
+
+		gen.writeStartObject();
+		gen.writeNumberField("refresh-interval", refreshInterval);
+		gen.writeNumberField("timezone-offset", timeZoneOffset);
+		gen.writeStringField("timezone-name", timeZoneName);
+		gen.writeStringField("flink-version", EnvironmentInformation.getVersion());
+
+		EnvironmentInformation.RevisionInformation revision = EnvironmentInformation.getRevisionInformation();
+		if (revision != null) {
+			gen.writeStringField("flink-revision", revision.commitId + " @ " + revision.commitDate);
+		}
+
+		gen.writeEndObject();
+
+		gen.close();
+
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphHolder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphHolder.java
new file mode 100644
index 0000000..8a47e50
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphHolder.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.WeakHashMap;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
+ *
+ * <p>The holder will cache the ExecutionGraph behind a weak reference, which will be cleared
+ * at some point once no one else is pointing to the ExecutionGraph.
+ * Note that while the holder runs in the same JVM as the JobManager or Archive, the reference should
+ * stay valid.
+ */
+public class ExecutionGraphHolder {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraphHolder.class);
+
+	private final Time timeout;
+
+	private final WeakHashMap<JobID, AccessExecutionGraph> cache = new WeakHashMap<>();
+
+	public ExecutionGraphHolder(Time timeout) {
+		this.timeout = checkNotNull(timeout);
+	}
+
+	/**
+	 * Retrieves the execution graph with {@link JobID} jid wrapped in {@link Optional} or
+	 * {@link Optional#empty()} if it cannot be found.
+	 *
+	 * @param jid jobID of the execution graph to be retrieved
+	 * @return Optional ExecutionGraph if it has been retrievable, empty if there has been no ExecutionGraph
+	 */
+	public CompletableFuture<Optional<AccessExecutionGraph>> getExecutionGraph(JobID jid, JobManagerGateway jobManagerGateway) {
+		AccessExecutionGraph cached = cache.get(jid);
+		if (cached != null) {
+			if (cached.getState() == JobStatus.SUSPENDED) {
+				cache.remove(jid);
+			} else {
+				return CompletableFuture.completedFuture(Optional.of(cached));
+			}
+		}
+
+		CompletableFuture<Optional<AccessExecutionGraph>> executionGraphFuture = jobManagerGateway.requestJob(jid, timeout);
+
+		executionGraphFuture.thenAcceptAsync(
+			optExecutionGraph ->
+				optExecutionGraph.ifPresent(executionGraph -> cache.put(jid, executionGraph)));
+
+		return executionGraphFuture;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandler.java
new file mode 100644
index 0000000..0a3b050
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandler.java
@@ -0,0 +1,106 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the aggregated user accumulators of a job.
+ */
+public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String JOB_ACCUMULATORS_REST_PATH = "/jobs/:jobid/accumulators";
+
+	public JobAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_ACCUMULATORS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createJobAccumulatorsJson(graph);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create job accumulators json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the JobAccumulatorsHandler.
+	 */
+	public static class JobAccumulatorsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createJobAccumulatorsJson(graph);
+			String path = JOB_ACCUMULATORS_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+
+	public static String createJobAccumulatorsJson(AccessExecutionGraph graph) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		StringifiedAccumulatorResult[] allAccumulators = graph.getAccumulatorResultsStringified();
+
+		gen.writeStartObject();
+
+		gen.writeArrayFieldStart("job-accumulators");
+		// empty for now
+		gen.writeEndArray();
+
+		gen.writeArrayFieldStart("user-task-accumulators");
+		for (StringifiedAccumulatorResult acc : allAccumulators) {
+			gen.writeStartObject();
+			gen.writeStringField("name", acc.getName());
+			gen.writeStringField("type", acc.getType());
+			gen.writeStringField("value", acc.getValue());
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandler.java
new file mode 100644
index 0000000..a194f30
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandler.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler for the CANCEL request.
+ */
+public class JobCancellationHandler extends AbstractJsonRequestHandler {
+
+	private static final String JOB_CONCELLATION_REST_PATH = "/jobs/:jobid/cancel";
+	private static final String JOB_CONCELLATION_YARN_REST_PATH = "/jobs/:jobid/yarn-cancel";
+
+	private final Time timeout;
+
+	public JobCancellationHandler(Executor executor, Time timeout) {
+		super(executor);
+		this.timeout = Preconditions.checkNotNull(timeout);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_CONCELLATION_REST_PATH, JOB_CONCELLATION_YARN_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid")));
+					if (jobManagerGateway != null) {
+						jobManagerGateway.cancelJob(jobId, timeout);
+						return "{}";
+					}
+					else {
+						throw new Exception("No connection to the leading JobManager.");
+					}
+				}
+				catch (Exception e) {
+					throw new FlinkFutureException("Failed to cancel the job with id: "  + pathParams.get("jobid"), e);
+				}
+			},
+			executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
new file mode 100644
index 0000000..23e94f5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlers.java
@@ -0,0 +1,427 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoint;
+import org.apache.flink.runtime.rest.NotFoundException;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Request handler for {@link CancelJobWithSavepoint} messages.
+ */
+public class JobCancellationWithSavepointHandlers {
+
+	private static final String CANCEL_WITH_SAVEPOINT_REST_PATH = "/jobs/:jobid/cancel-with-savepoint";
+	private static final String CANCEL_WITH_SAVEPOINT_DIRECTORY_REST_PATH = "/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory";
+
+	/** URL for in-progress cancellations. */
+	private static final String CANCELLATION_IN_PROGRESS_REST_PATH = "/jobs/:jobid/cancel-with-savepoint/in-progress/:requestId";
+
+	/** Encodings for String. */
+	private static final Charset ENCODING = ConfigConstants.DEFAULT_CHARSET;
+
+	/** Shared lock between Trigger and In-Progress handlers. */
+	private final Object lock = new Object();
+
+	/** In-Progress requests. */
+	private final Map<JobID, Long> inProgress = new HashMap<>();
+
+	/** Succeeded/failed request. Either String or Throwable. */
+	private final Map<Long, Object> completed = new HashMap<>();
+
+	/** Atomic request counter. */
+	private long requestCounter;
+
+	/** Handler for trigger requests. */
+	private final TriggerHandler triggerHandler;
+
+	/** Handler for in-progress requests. */
+	private final InProgressHandler inProgressHandler;
+
+	/** Default savepoint directory. */
+	private final String defaultSavepointDirectory;
+
+	public JobCancellationWithSavepointHandlers(
+			ExecutionGraphHolder currentGraphs,
+			Executor executor) {
+		this(currentGraphs, executor, null);
+	}
+
+	public JobCancellationWithSavepointHandlers(
+			ExecutionGraphHolder currentGraphs,
+			Executor executor,
+			@Nullable String defaultSavepointDirectory) {
+
+		this.triggerHandler = new TriggerHandler(currentGraphs, executor);
+		this.inProgressHandler = new InProgressHandler();
+		this.defaultSavepointDirectory = defaultSavepointDirectory;
+	}
+
+	public TriggerHandler getTriggerHandler() {
+		return triggerHandler;
+	}
+
+	public InProgressHandler getInProgressHandler() {
+		return inProgressHandler;
+	}
+
+	// ------------------------------------------------------------------------
+	// New requests
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Handler for triggering a {@link CancelJobWithSavepoint} message.
+	 */
+	class TriggerHandler implements RequestHandler {
+
+		/** Current execution graphs. */
+		private final ExecutionGraphHolder currentGraphs;
+
+		/** Execution context for futures. */
+		private final Executor executor;
+
+		public TriggerHandler(ExecutionGraphHolder currentGraphs, Executor executor) {
+			this.currentGraphs = checkNotNull(currentGraphs);
+			this.executor = checkNotNull(executor);
+		}
+
+		@Override
+		public String[] getPaths() {
+			return new String[]{CANCEL_WITH_SAVEPOINT_REST_PATH, CANCEL_WITH_SAVEPOINT_DIRECTORY_REST_PATH};
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public CompletableFuture<FullHttpResponse> handleRequest(
+				Map<String, String> pathParams,
+				Map<String, String> queryParams,
+				JobManagerGateway jobManagerGateway) {
+
+			if (jobManagerGateway != null) {
+				JobID jobId = JobID.fromHexString(pathParams.get("jobid"));
+				final CompletableFuture<Optional<AccessExecutionGraph>> graphFuture;
+
+				graphFuture = currentGraphs.getExecutionGraph(jobId, jobManagerGateway);
+
+				return graphFuture.thenApplyAsync(
+					(Optional<AccessExecutionGraph> optGraph) -> {
+						final AccessExecutionGraph graph = optGraph.orElseThrow(
+							() -> new FlinkFutureException(
+								new NotFoundException("Could not find ExecutionGraph with jobId " + jobId + '.')));
+
+						CheckpointCoordinator coord = graph.getCheckpointCoordinator();
+						if (coord == null) {
+							throw new FlinkFutureException(new Exception("Cannot find CheckpointCoordinator for job."));
+						}
+
+						String targetDirectory = pathParams.get("targetDirectory");
+						if (targetDirectory == null) {
+							if (defaultSavepointDirectory == null) {
+								throw new IllegalStateException("No savepoint directory configured. " +
+									"You can either specify a directory when triggering this savepoint or " +
+									"configure a cluster-wide default via key '" +
+									CoreOptions.SAVEPOINT_DIRECTORY.key() + "'.");
+							} else {
+								targetDirectory = defaultSavepointDirectory;
+							}
+						}
+
+						try {
+							return handleNewRequest(jobManagerGateway, jobId, targetDirectory, coord.getCheckpointTimeout());
+						} catch (IOException e) {
+							throw new FlinkFutureException("Could not cancel job with savepoint.", e);
+						}
+					}, executor);
+			} else {
+				return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
+			}
+		}
+
+		@SuppressWarnings("unchecked")
+		private FullHttpResponse handleNewRequest(JobManagerGateway jobManagerGateway, final JobID jobId, String targetDirectory, long checkpointTimeout) throws IOException {
+			// Check whether a request exists
+			final long requestId;
+			final boolean isNewRequest;
+			synchronized (lock) {
+				if (inProgress.containsKey(jobId)) {
+					requestId = inProgress.get(jobId);
+					isNewRequest = false;
+				} else {
+					requestId = ++requestCounter;
+					inProgress.put(jobId, requestId);
+					isNewRequest = true;
+				}
+			}
+
+			if (isNewRequest) {
+				boolean success = false;
+
+				try {
+					// Trigger cancellation
+					CompletableFuture<String> cancelJobFuture = jobManagerGateway
+						.cancelJobWithSavepoint(jobId, targetDirectory, Time.milliseconds(checkpointTimeout));
+
+					cancelJobFuture.whenCompleteAsync(
+						(String path, Throwable throwable) -> {
+							try {
+								if (throwable != null) {
+									completed.put(requestId, throwable);
+								} else {
+									completed.put(requestId, path);
+								}
+							} finally {
+								inProgress.remove(jobId);
+							}
+						}, executor);
+
+					success = true;
+				} finally {
+					synchronized (lock) {
+						if (!success) {
+							inProgress.remove(jobId);
+						}
+					}
+				}
+			}
+
+			// In-progress location
+			String location = CANCELLATION_IN_PROGRESS_REST_PATH
+					.replace(":jobid", jobId.toString())
+					.replace(":requestId", Long.toString(requestId));
+
+			// Accepted response
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+			gen.writeStartObject();
+			gen.writeStringField("status", "accepted");
+			gen.writeNumberField("request-id", requestId);
+			gen.writeStringField("location", location);
+			gen.writeEndObject();
+			gen.close();
+
+			String json = writer.toString();
+			byte[] bytes = json.getBytes(ENCODING);
+
+			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
+					HttpVersion.HTTP_1_1,
+					HttpResponseStatus.ACCEPTED,
+					Unpooled.wrappedBuffer(bytes));
+
+			response.headers().set(HttpHeaders.Names.LOCATION, location);
+
+			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
+			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
+
+			FullHttpResponse accepted = response;
+
+			return accepted;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	// In-progress requests
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Handler for in-progress cancel with savepoint operations.
+	 */
+	class InProgressHandler implements RequestHandler {
+
+		/** The number of recent checkpoints whose IDs are remembered. */
+		private static final int NUM_GHOST_REQUEST_IDS = 16;
+
+		/** Remember some recently completed. */
+		private final ArrayDeque<Tuple2<Long, Object>> recentlyCompleted = new ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
+
+		@Override
+		public String[] getPaths() {
+			return new String[]{CANCELLATION_IN_PROGRESS_REST_PATH};
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+			JobID jobId = JobID.fromHexString(pathParams.get("jobid"));
+			long requestId = Long.parseLong(pathParams.get("requestId"));
+
+			return CompletableFuture.supplyAsync(
+				() -> {
+					try {
+						synchronized (lock) {
+							Object result = completed.remove(requestId);
+
+							if (result != null) {
+								// Add to recent history
+								recentlyCompleted.add(new Tuple2<>(requestId, result));
+								if (recentlyCompleted.size() > NUM_GHOST_REQUEST_IDS) {
+									recentlyCompleted.remove();
+								}
+
+								if (result.getClass() == String.class) {
+									String savepointPath = (String) result;
+									return createSuccessResponse(requestId, savepointPath);
+								} else {
+									Throwable cause = (Throwable) result;
+									return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage());
+								}
+							} else {
+								// Check in-progress
+								Long inProgressRequestId = inProgress.get(jobId);
+								if (inProgressRequestId != null) {
+									// Sanity check
+									if (inProgressRequestId == requestId) {
+										return createInProgressResponse(requestId);
+									} else {
+										String msg = "Request ID does not belong to JobID";
+										return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, msg);
+									}
+								}
+
+								// Check recent history
+								for (Tuple2<Long, Object> recent : recentlyCompleted) {
+									if (recent.f0 == requestId) {
+										if (recent.f1.getClass() == String.class) {
+											String savepointPath = (String) recent.f1;
+											return createSuccessResponse(requestId, savepointPath);
+										} else {
+											Throwable cause = (Throwable) recent.f1;
+											return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage());
+										}
+									}
+								}
+
+								return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, "Unknown job/request ID");
+							}
+						}
+					} catch (Exception e) {
+						throw new FlinkFutureException("Could not handle in progress request.", e);
+					}
+				});
+		}
+
+		private FullHttpResponse createSuccessResponse(long requestId, String savepointPath) throws IOException {
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+			gen.writeStartObject();
+
+			gen.writeStringField("status", "success");
+			gen.writeNumberField("request-id", requestId);
+			gen.writeStringField("savepoint-path", savepointPath);
+
+			gen.writeEndObject();
+			gen.close();
+
+			String json = writer.toString();
+			byte[] bytes = json.getBytes(ENCODING);
+
+			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
+					HttpVersion.HTTP_1_1,
+					HttpResponseStatus.CREATED,
+					Unpooled.wrappedBuffer(bytes));
+
+			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
+			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
+
+			return response;
+		}
+
+		private FullHttpResponse createInProgressResponse(long requestId) throws IOException {
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+			gen.writeStartObject();
+
+			gen.writeStringField("status", "in-progress");
+			gen.writeNumberField("request-id", requestId);
+
+			gen.writeEndObject();
+			gen.close();
+
+			String json = writer.toString();
+			byte[] bytes = json.getBytes(ENCODING);
+
+			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
+					HttpVersion.HTTP_1_1,
+					HttpResponseStatus.ACCEPTED,
+					Unpooled.wrappedBuffer(bytes));
+
+			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
+			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
+
+			return response;
+		}
+
+		private FullHttpResponse createFailureResponse(HttpResponseStatus code, long requestId, String errMsg) throws IOException {
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+			gen.writeStartObject();
+
+			gen.writeStringField("status", "failed");
+			gen.writeNumberField("request-id", requestId);
+			gen.writeStringField("cause", errMsg);
+
+			gen.writeEndObject();
+			gen.close();
+
+			String json = writer.toString();
+			byte[] bytes = json.getBytes(ENCODING);
+
+			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
+					HttpVersion.HTTP_1_1,
+					code,
+					Unpooled.wrappedBuffer(bytes));
+
+			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
+			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
+
+			return response;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandler.java
new file mode 100644
index 0000000..bb1cf8f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandler.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.ArchivedExecutionConfig;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the execution config of a job.
+ */
+public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String JOB_CONFIG_REST_PATH = "/jobs/:jobid/config";
+
+	public JobConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_CONFIG_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createJobConfigJson(graph);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not write job config json.", e);
+				}
+			},
+			executor);
+
+	}
+
+	/**
+	 * Archivist for the JobConfigHandler.
+	 */
+	public static class JobConfigJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createJobConfigJson(graph);
+			String path = JOB_CONFIG_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+
+	public static String createJobConfigJson(AccessExecutionGraph graph) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+		gen.writeStringField("jid", graph.getJobID().toString());
+		gen.writeStringField("name", graph.getJobName());
+
+		final ArchivedExecutionConfig summary = graph.getArchivedExecutionConfig();
+
+		if (summary != null) {
+			gen.writeObjectFieldStart("execution-config");
+
+			gen.writeStringField("execution-mode", summary.getExecutionMode());
+
+			gen.writeStringField("restart-strategy", summary.getRestartStrategyDescription());
+			gen.writeNumberField("job-parallelism", summary.getParallelism());
+			gen.writeBooleanField("object-reuse-mode", summary.getObjectReuseEnabled());
+
+			Map<String, String> ucVals = summary.getGlobalJobParameters();
+			if (ucVals != null) {
+				gen.writeObjectFieldStart("user-config");
+
+				for (Map.Entry<String, String> ucVal : ucVals.entrySet()) {
+					gen.writeStringField(ucVal.getKey(), ucVal.getValue());
+				}
+
+				gen.writeEndObject();
+			}
+
+			gen.writeEndObject();
+		}
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandler.java
new file mode 100644
index 0000000..dd6aee8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandler.java
@@ -0,0 +1,224 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns details about a job. This includes:
+ * <ul>
+ *     <li>Dataflow plan</li>
+ *     <li>id, name, and current status</li>
+ *     <li>start time, end time, duration</li>
+ *     <li>number of job vertices in each state (pending, running, finished, failed)</li>
+ *     <li>info about job vertices, including runtime, status, I/O bytes and records, subtasks in each status</li>
+ * </ul>
+ */
+public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String JOB_DETAILS_REST_PATH = "/jobs/:jobid";
+	private static final String JOB_DETAILS_VERTICES_REST_PATH = "/jobs/:jobid/vertices";
+
+	private final MetricFetcher fetcher;
+
+	public JobDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
+		super(executionGraphHolder, executor);
+		this.fetcher = fetcher;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_DETAILS_REST_PATH, JOB_DETAILS_VERTICES_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createJobDetailsJson(graph, fetcher);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create job details json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the JobDetailsHandler.
+	 */
+	public static class JobDetailsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createJobDetailsJson(graph, null);
+			String path1 = JOB_DETAILS_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			String path2 = JOB_DETAILS_VERTICES_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			Collection<ArchivedJson> archives = new ArrayList<>();
+			archives.add(new ArchivedJson(path1, json));
+			archives.add(new ArchivedJson(path2, json));
+			return archives;
+		}
+	}
+
+	public static String createJobDetailsJson(AccessExecutionGraph graph, @Nullable MetricFetcher fetcher) throws IOException {
+		final StringWriter writer = new StringWriter();
+		final JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		final long now = System.currentTimeMillis();
+
+		gen.writeStartObject();
+
+		// basic info
+		gen.writeStringField("jid", graph.getJobID().toString());
+		gen.writeStringField("name", graph.getJobName());
+		gen.writeBooleanField("isStoppable", graph.isStoppable());
+		gen.writeStringField("state", graph.getState().name());
+
+		// times and duration
+		final long jobStartTime = graph.getStatusTimestamp(JobStatus.CREATED);
+		final long jobEndTime = graph.getState().isGloballyTerminalState() ?
+				graph.getStatusTimestamp(graph.getState()) : -1L;
+		gen.writeNumberField("start-time", jobStartTime);
+		gen.writeNumberField("end-time", jobEndTime);
+		gen.writeNumberField("duration", (jobEndTime > 0 ? jobEndTime : now) - jobStartTime);
+		gen.writeNumberField("now", now);
+
+		// timestamps
+		gen.writeObjectFieldStart("timestamps");
+		for (JobStatus status : JobStatus.values()) {
+			gen.writeNumberField(status.name(), graph.getStatusTimestamp(status));
+		}
+		gen.writeEndObject();
+
+		// job vertices
+		int[] jobVerticesPerState = new int[ExecutionState.values().length];
+		gen.writeArrayFieldStart("vertices");
+
+		for (AccessExecutionJobVertex ejv : graph.getVerticesTopologically()) {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			long startTime = Long.MAX_VALUE;
+			long endTime = 0;
+			boolean allFinished = true;
+
+			for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
+				final ExecutionState state = vertex.getExecutionState();
+				tasksPerState[state.ordinal()]++;
+
+				// take the earliest start time
+				long started = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
+				if (started > 0) {
+					startTime = Math.min(startTime, started);
+				}
+
+				allFinished &= state.isTerminal();
+				endTime = Math.max(endTime, vertex.getStateTimestamp(state));
+			}
+
+			long duration;
+			if (startTime < Long.MAX_VALUE) {
+				if (allFinished) {
+					duration = endTime - startTime;
+				}
+				else {
+					endTime = -1L;
+					duration = now - startTime;
+				}
+			}
+			else {
+				startTime = -1L;
+				endTime = -1L;
+				duration = -1L;
+			}
+
+			ExecutionState jobVertexState =
+					ExecutionJobVertex.getAggregateJobVertexState(tasksPerState, ejv.getParallelism());
+			jobVerticesPerState[jobVertexState.ordinal()]++;
+
+			gen.writeStartObject();
+			gen.writeStringField("id", ejv.getJobVertexId().toString());
+			gen.writeStringField("name", ejv.getName());
+			gen.writeNumberField("parallelism", ejv.getParallelism());
+			gen.writeStringField("status", jobVertexState.name());
+
+			gen.writeNumberField("start-time", startTime);
+			gen.writeNumberField("end-time", endTime);
+			gen.writeNumberField("duration", duration);
+
+			gen.writeObjectFieldStart("tasks");
+			for (ExecutionState state : ExecutionState.values()) {
+				gen.writeNumberField(state.name(), tasksPerState[state.ordinal()]);
+			}
+			gen.writeEndObject();
+
+			MutableIOMetrics counts = new MutableIOMetrics();
+
+			for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
+				counts.addIOMetrics(
+					vertex.getCurrentExecutionAttempt(),
+					fetcher,
+					graph.getJobID().toString(),
+					ejv.getJobVertexId().toString());
+			}
+
+			counts.writeIOMetricsAsJson(gen);
+
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeObjectFieldStart("status-counts");
+		for (ExecutionState state : ExecutionState.values()) {
+			gen.writeNumberField(state.name(), jobVerticesPerState[state.ordinal()]);
+		}
+		gen.writeEndObject();
+
+		gen.writeFieldName("plan");
+		gen.writeRawValue(graph.getJsonPlan());
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}


[16/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
[FLINK-7531] Move Flink legacy rest handler to flink-runtime

Move metrics handlers under o.a.f.runtime.webmonitor.handlers

Move StaticFileServerHandler under o.a.f.runtime.webmonitor.files

This closes #4600.


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

Branch: refs/heads/master
Commit: 4fc019a96a08446d7ba5f57664904abcd585e31c
Parents: 3277010
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Aug 18 09:52:30 2017 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Wed Sep 20 00:40:24 2017 +0200

----------------------------------------------------------------------
 .../webmonitor/BackPressureStatsTracker.java    | 334 --------------
 .../webmonitor/ExecutionGraphHolder.java        |  86 ----
 .../runtime/webmonitor/NotFoundException.java   |  32 --
 .../webmonitor/OperatorBackPressureStats.java   | 126 ------
 .../webmonitor/RuntimeMonitorHandler.java       |   4 +-
 .../runtime/webmonitor/StackTraceSample.java    | 119 -----
 .../webmonitor/StackTraceSampleCoordinator.java | 392 -----------------
 .../flink/runtime/webmonitor/WebHandler.java    |  32 --
 .../runtime/webmonitor/WebRuntimeMonitor.java   |  82 ++--
 .../files/StaticFileServerHandler.java          | 363 ---------------
 .../AbstractExecutionGraphRequestHandler.java   |  80 ----
 .../AbstractJobVertexRequestHandler.java        |  72 ---
 .../handlers/AbstractJsonRequestHandler.java    |  87 ----
 .../AbstractSubtaskAttemptRequestHandler.java   |  78 ----
 .../handlers/AbstractSubtaskRequestHandler.java |  66 ---
 .../handlers/ClusterOverviewHandler.java        | 105 -----
 .../handlers/ConstantTextHandler.java           |  57 ---
 .../handlers/CurrentJobIdsHandler.java          | 112 -----
 .../handlers/CurrentJobsOverviewHandler.java    | 182 --------
 .../handlers/DashboardConfigHandler.java        |  90 ----
 .../handlers/JarAccessDeniedHandler.java        |   1 +
 .../webmonitor/handlers/JarActionHandler.java   |   2 +
 .../webmonitor/handlers/JarDeleteHandler.java   |   2 +
 .../webmonitor/handlers/JarListHandler.java     |   2 +
 .../webmonitor/handlers/JarPlanHandler.java     |   1 +
 .../webmonitor/handlers/JarRunHandler.java      |   1 +
 .../webmonitor/handlers/JarUploadHandler.java   |   1 +
 .../handlers/JobAccumulatorsHandler.java        | 107 -----
 .../handlers/JobCancellationHandler.java        |  72 ---
 .../JobCancellationWithSavepointHandlers.java   | 428 ------------------
 .../webmonitor/handlers/JobConfigHandler.java   | 119 -----
 .../webmonitor/handlers/JobDetailsHandler.java  | 225 ----------
 .../handlers/JobExceptionsHandler.java          | 137 ------
 .../handlers/JobManagerConfigHandler.java       |  87 ----
 .../webmonitor/handlers/JobPlanHandler.java     |  67 ---
 .../webmonitor/handlers/JobStoppingHandler.java |  72 ---
 .../handlers/JobVertexAccumulatorsHandler.java  | 113 -----
 .../handlers/JobVertexBackPressureHandler.java  | 147 -------
 .../handlers/JobVertexDetailsHandler.java       | 160 -------
 .../handlers/JobVertexTaskManagersHandler.java  | 211 ---------
 .../webmonitor/handlers/JsonFactory.java        |  35 --
 .../webmonitor/handlers/RequestHandler.java     |  56 ---
 .../handlers/RequestHandlerException.java       |  31 --
 .../SubtaskCurrentAttemptDetailsHandler.java    |  49 ---
 ...taskExecutionAttemptAccumulatorsHandler.java | 134 ------
 .../SubtaskExecutionAttemptDetailsHandler.java  | 167 -------
 .../SubtasksAllAccumulatorsHandler.java         | 131 ------
 .../handlers/SubtasksTimesHandler.java          | 141 ------
 .../handlers/TaskManagerLogHandler.java         | 335 --------------
 .../handlers/TaskManagersHandler.java           | 205 ---------
 .../checkpoints/CheckpointConfigHandler.java    | 120 -----
 .../checkpoints/CheckpointStatsCache.java       |  81 ----
 .../CheckpointStatsDetailsHandler.java          | 203 ---------
 .../CheckpointStatsDetailsSubtasksHandler.java  | 234 ----------
 .../checkpoints/CheckpointStatsHandler.java     | 277 ------------
 .../webmonitor/history/HistoryServer.java       |   2 +-
 .../history/HistoryServerArchiveFetcher.java    |   2 +-
 .../HistoryServerStaticFileServerHandler.java   |   2 +-
 .../metrics/AbstractMetricsHandler.java         | 139 ------
 .../metrics/JobManagerMetricsHandler.java       |  57 ---
 .../webmonitor/metrics/JobMetricsHandler.java   |  55 ---
 .../metrics/JobVertexMetricsHandler.java        |  57 ---
 .../webmonitor/metrics/MetricFetcher.java       | 211 ---------
 .../runtime/webmonitor/metrics/MetricStore.java | 305 -------------
 .../metrics/TaskManagerMetricsHandler.java      |  59 ---
 .../webmonitor/utils/MutableIOMetrics.java      | 109 -----
 .../BackPressureStatsTrackerITCase.java         | 332 --------------
 .../BackPressureStatsTrackerTest.java           | 192 --------
 .../StackTraceSampleCoordinatorITCase.java      | 203 ---------
 .../StackTraceSampleCoordinatorTest.java        | 441 -------------------
 .../runtime/webmonitor/files/MimeTypesTest.java |  75 ----
 .../handlers/ClusterOverviewHandlerTest.java    |  38 --
 .../handlers/CurrentJobIdsHandlerTest.java      |  38 --
 .../CurrentJobsOverviewHandlerTest.java         | 121 -----
 .../handlers/DashboardConfigHandlerTest.java    |  59 ---
 .../handlers/HandlerRedirectUtilsTest.java      |  74 ----
 .../handlers/JarActionHandlerTest.java          |  13 +-
 .../handlers/JobAccumulatorsHandlerTest.java    |  83 ----
 .../handlers/JobCancellationHandlerTest.java    |  44 --
 ...obCancellationWithSavepointHandlersTest.java | 334 --------------
 .../handlers/JobConfigHandlerTest.java          |  92 ----
 .../handlers/JobDetailsHandlerTest.java         | 169 -------
 .../handlers/JobExceptionsHandlerTest.java      | 101 -----
 .../handlers/JobManagerConfigHandlerTest.java   |  37 --
 .../webmonitor/handlers/JobPlanHandlerTest.java |  60 ---
 .../handlers/JobStoppingHandlerTest.java        |  45 --
 .../JobVertexAccumulatorsHandlerTest.java       |  85 ----
 .../JobVertexBackPressureHandlerTest.java       | 211 ---------
 .../handlers/JobVertexDetailsHandlerTest.java   | 109 -----
 .../JobVertexTaskManagersHandlerTest.java       | 132 ------
 ...SubtaskCurrentAttemptDetailsHandlerTest.java |  40 --
 ...ExecutionAttemptAccumulatorsHandlerTest.java |  91 ----
 ...btaskExecutionAttemptDetailsHandlerTest.java | 109 -----
 .../SubtasksAllAccumulatorsHandlerTest.java     |  97 ----
 .../handlers/SubtasksTimesHandlerTest.java      | 103 -----
 .../handlers/TaskManagerLogHandlerTest.java     | 149 -------
 .../handlers/TaskManagersHandlerTest.java       |  44 --
 .../CheckpointConfigHandlerTest.java            | 195 --------
 .../checkpoints/CheckpointStatsCacheTest.java   |  71 ---
 .../CheckpointStatsDetailsHandlerTest.java      | 358 ---------------
 .../checkpoints/CheckpointStatsHandlerTest.java | 432 ------------------
 ...heckpointStatsSubtaskDetailsHandlerTest.java | 389 ----------------
 .../webmonitor/history/FsJobArchivistTest.java  |   2 +-
 .../webmonitor/history/HistoryServerTest.java   |   2 +-
 .../metrics/AbstractMetricsHandlerTest.java     | 172 --------
 .../metrics/JobManagerMetricsHandlerTest.java   |  84 ----
 .../metrics/JobMetricsHandlerTest.java          |  86 ----
 .../metrics/JobVertexMetricsHandlerTest.java    |  90 ----
 .../webmonitor/metrics/MetricFetcherTest.java   | 195 --------
 .../webmonitor/metrics/MetricStoreTest.java     |  88 ----
 .../metrics/TaskManagerMetricsHandlerTest.java  |  86 ----
 .../utils/ArchivedExecutionBuilder.java         | 150 -------
 .../utils/ArchivedExecutionConfigBuilder.java   |  71 ---
 .../utils/ArchivedExecutionGraphBuilder.java    | 140 ------
 .../ArchivedExecutionJobVertexBuilder.java      |  84 ----
 .../utils/ArchivedExecutionVertexBuilder.java   |  73 ---
 .../utils/ArchivedJobGenerationUtils.java       | 164 -------
 .../flink/runtime/rest/NotFoundException.java   |  32 ++
 .../flink/runtime/rest/handler/WebHandler.java  |  32 ++
 .../AbstractExecutionGraphRequestHandler.java   |  79 ++++
 .../legacy/AbstractJobVertexRequestHandler.java |  71 +++
 .../legacy/AbstractJsonRequestHandler.java      |  88 ++++
 .../AbstractSubtaskAttemptRequestHandler.java   |  77 ++++
 .../legacy/AbstractSubtaskRequestHandler.java   |  65 +++
 .../handler/legacy/ClusterOverviewHandler.java  | 105 +++++
 .../handler/legacy/ConstantTextHandler.java     |  57 +++
 .../handler/legacy/CurrentJobIdsHandler.java    | 112 +++++
 .../legacy/CurrentJobsOverviewHandler.java      | 182 ++++++++
 .../handler/legacy/DashboardConfigHandler.java  |  90 ++++
 .../handler/legacy/ExecutionGraphHolder.java    |  82 ++++
 .../handler/legacy/JobAccumulatorsHandler.java  | 106 +++++
 .../handler/legacy/JobCancellationHandler.java  |  72 +++
 .../JobCancellationWithSavepointHandlers.java   | 427 ++++++++++++++++++
 .../rest/handler/legacy/JobConfigHandler.java   | 118 +++++
 .../rest/handler/legacy/JobDetailsHandler.java  | 224 ++++++++++
 .../handler/legacy/JobExceptionsHandler.java    | 136 ++++++
 .../handler/legacy/JobManagerConfigHandler.java |  87 ++++
 .../rest/handler/legacy/JobPlanHandler.java     |  66 +++
 .../rest/handler/legacy/JobStoppingHandler.java |  72 +++
 .../legacy/JobVertexAccumulatorsHandler.java    | 112 +++++
 .../legacy/JobVertexBackPressureHandler.java    | 145 ++++++
 .../handler/legacy/JobVertexDetailsHandler.java | 159 +++++++
 .../legacy/JobVertexTaskManagersHandler.java    | 210 +++++++++
 .../rest/handler/legacy/JsonFactory.java        |  35 ++
 .../rest/handler/legacy/RequestHandler.java     |  56 +++
 .../handler/legacy/RequestHandlerException.java |  31 ++
 .../SubtaskCurrentAttemptDetailsHandler.java    |  48 ++
 ...taskExecutionAttemptAccumulatorsHandler.java | 133 ++++++
 .../SubtaskExecutionAttemptDetailsHandler.java  | 166 +++++++
 .../legacy/SubtasksAllAccumulatorsHandler.java  | 130 ++++++
 .../handler/legacy/SubtasksTimesHandler.java    | 140 ++++++
 .../handler/legacy/TaskManagerLogHandler.java   | 335 ++++++++++++++
 .../handler/legacy/TaskManagersHandler.java     | 205 +++++++++
 .../backpressure/BackPressureStatsTracker.java  | 333 ++++++++++++++
 .../backpressure/OperatorBackPressureStats.java | 126 ++++++
 .../legacy/backpressure/StackTraceSample.java   | 119 +++++
 .../StackTraceSampleCoordinator.java            | 392 +++++++++++++++++
 .../checkpoints/CheckpointConfigHandler.java    | 120 +++++
 .../checkpoints/CheckpointStatsCache.java       |  81 ++++
 .../CheckpointStatsDetailsHandler.java          | 203 +++++++++
 .../CheckpointStatsDetailsSubtasksHandler.java  | 233 ++++++++++
 .../checkpoints/CheckpointStatsHandler.java     | 277 ++++++++++++
 .../legacy/files/StaticFileServerHandler.java   | 363 +++++++++++++++
 .../legacy/metrics/AbstractMetricsHandler.java  | 139 ++++++
 .../metrics/JobManagerMetricsHandler.java       |  57 +++
 .../legacy/metrics/JobMetricsHandler.java       |  55 +++
 .../legacy/metrics/JobVertexMetricsHandler.java |  57 +++
 .../handler/legacy/metrics/MetricFetcher.java   | 211 +++++++++
 .../handler/legacy/metrics/MetricStore.java     | 305 +++++++++++++
 .../metrics/TaskManagerMetricsHandler.java      |  59 +++
 .../rest/handler/util/MutableIOMetrics.java     | 109 +++++
 .../legacy/ClusterOverviewHandlerTest.java      |  38 ++
 .../legacy/CurrentJobIdsHandlerTest.java        |  38 ++
 .../legacy/CurrentJobsOverviewHandlerTest.java  | 121 +++++
 .../legacy/DashboardConfigHandlerTest.java      |  59 +++
 .../legacy/HandlerRedirectUtilsTest.java        |  74 ++++
 .../legacy/JobAccumulatorsHandlerTest.java      |  82 ++++
 .../legacy/JobCancellationHandlerTest.java      |  44 ++
 ...obCancellationWithSavepointHandlersTest.java | 333 ++++++++++++++
 .../handler/legacy/JobConfigHandlerTest.java    |  91 ++++
 .../handler/legacy/JobDetailsHandlerTest.java   | 168 +++++++
 .../legacy/JobExceptionsHandlerTest.java        | 100 +++++
 .../legacy/JobManagerConfigHandlerTest.java     |  37 ++
 .../rest/handler/legacy/JobPlanHandlerTest.java |  59 +++
 .../handler/legacy/JobStoppingHandlerTest.java  |  45 ++
 .../JobVertexAccumulatorsHandlerTest.java       |  84 ++++
 .../JobVertexBackPressureHandlerTest.java       | 209 +++++++++
 .../legacy/JobVertexDetailsHandlerTest.java     | 108 +++++
 .../JobVertexTaskManagersHandlerTest.java       | 132 ++++++
 ...SubtaskCurrentAttemptDetailsHandlerTest.java |  40 ++
 ...ExecutionAttemptAccumulatorsHandlerTest.java |  91 ++++
 ...btaskExecutionAttemptDetailsHandlerTest.java | 109 +++++
 .../SubtasksAllAccumulatorsHandlerTest.java     |  97 ++++
 .../legacy/SubtasksTimesHandlerTest.java        | 103 +++++
 .../legacy/TaskManagerLogHandlerTest.java       | 149 +++++++
 .../handler/legacy/TaskManagersHandlerTest.java |  44 ++
 .../BackPressureStatsTrackerITCase.java         | 329 ++++++++++++++
 .../BackPressureStatsTrackerTest.java           | 185 ++++++++
 .../StackTraceSampleCoordinatorITCase.java      | 203 +++++++++
 .../StackTraceSampleCoordinatorTest.java        | 432 ++++++++++++++++++
 .../CheckpointConfigHandlerTest.java            | 195 ++++++++
 .../checkpoints/CheckpointStatsCacheTest.java   |  71 +++
 .../CheckpointStatsDetailsHandlerTest.java      | 358 +++++++++++++++
 .../checkpoints/CheckpointStatsHandlerTest.java | 432 ++++++++++++++++++
 ...heckpointStatsSubtaskDetailsHandlerTest.java | 389 ++++++++++++++++
 .../handler/legacy/files/MimeTypesTest.java     |  75 ++++
 .../metrics/AbstractMetricsHandlerTest.java     | 172 ++++++++
 .../metrics/JobManagerMetricsHandlerTest.java   |  84 ++++
 .../legacy/metrics/JobMetricsHandlerTest.java   |  86 ++++
 .../metrics/JobVertexMetricsHandlerTest.java    |  90 ++++
 .../legacy/metrics/MetricFetcherTest.java       | 195 ++++++++
 .../handler/legacy/metrics/MetricStoreTest.java |  88 ++++
 .../metrics/TaskManagerMetricsHandlerTest.java  |  86 ++++
 .../legacy/utils/ArchivedExecutionBuilder.java  | 150 +++++++
 .../utils/ArchivedExecutionConfigBuilder.java   |  71 +++
 .../utils/ArchivedExecutionGraphBuilder.java    | 140 ++++++
 .../ArchivedExecutionJobVertexBuilder.java      |  84 ++++
 .../utils/ArchivedExecutionVertexBuilder.java   |  73 +++
 .../utils/ArchivedJobGenerationUtils.java       | 164 +++++++
 219 files changed, 14198 insertions(+), 14237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTracker.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTracker.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTracker.java
deleted file mode 100644
index 5e4e63a..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTracker.java
+++ /dev/null
@@ -1,334 +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.webmonitor;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-
-import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
-import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
-import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-
-import scala.Option;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Back pressure statistics tracker.
- *
- * <p>Back pressure is determined by sampling running tasks. If a task is
- * slowed down by back pressure it will be stuck in memory requests to a
- * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}.
- *
- * <p>The back pressured stack traces look like this:
- *
- * <pre>
- * java.lang.Object.wait(Native Method)
- * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
- * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) <--- BLOCKING
- * request
- * [...]
- * </pre>
- */
-public class BackPressureStatsTracker {
-
-	private static final Logger LOG = LoggerFactory.getLogger(BackPressureStatsTracker.class);
-
-	/** Maximum stack trace depth for samples. */
-	static final int MAX_STACK_TRACE_DEPTH = 3;
-
-	/** Expected class name for back pressure indicating stack trace element. */
-	static final String EXPECTED_CLASS_NAME = "org.apache.flink.runtime.io.network.buffer.LocalBufferPool";
-
-	/** Expected method name for back pressure indicating stack trace element. */
-	static final String EXPECTED_METHOD_NAME = "requestBufferBlocking";
-
-	/** Lock guarding trigger operations. */
-	private final Object lock = new Object();
-
-	/* Stack trace sample coordinator. */
-	private final StackTraceSampleCoordinator coordinator;
-
-	/**
-	 * Completed stats. Important: Job vertex IDs need to be scoped by job ID,
-	 * because they are potentially constant across runs messing up the cached
-	 * data.
-	 */
-	private final Cache<ExecutionJobVertex, OperatorBackPressureStats> operatorStatsCache;
-
-	/** Pending in progress stats. Important: Job vertex IDs need to be scoped
-	 * by job ID, because they are potentially constant across runs messing up
-	 * the cached data.*/
-	private final Set<ExecutionJobVertex> pendingStats = new HashSet<>();
-
-	/** Cleanup interval for completed stats cache. */
-	private final int cleanUpInterval;
-
-	private final int numSamples;
-
-	private final Time delayBetweenSamples;
-
-	/** Flag indicating whether the stats tracker has been shut down. */
-	private boolean shutDown;
-
-	/**
-	 * Creates a back pressure statistics tracker.
-	 *
-	 * @param cleanUpInterval     Clean up interval for completed stats.
-	 * @param numSamples          Number of stack trace samples when determining back pressure.
-	 * @param delayBetweenSamples Delay between samples when determining back pressure.
-	 */
-	public BackPressureStatsTracker(
-			StackTraceSampleCoordinator coordinator,
-			int cleanUpInterval,
-			int numSamples,
-			Time delayBetweenSamples) {
-
-		this.coordinator = checkNotNull(coordinator, "Stack trace sample coordinator");
-
-		checkArgument(cleanUpInterval >= 0, "Clean up interval");
-		this.cleanUpInterval = cleanUpInterval;
-
-		checkArgument(numSamples >= 1, "Number of samples");
-		this.numSamples = numSamples;
-
-		this.delayBetweenSamples = checkNotNull(delayBetweenSamples, "Delay between samples");
-
-		this.operatorStatsCache = CacheBuilder.newBuilder()
-				.concurrencyLevel(1)
-				.expireAfterAccess(cleanUpInterval, TimeUnit.MILLISECONDS)
-				.build();
-	}
-
-	/** Cleanup interval for completed stats cache. */
-	public long getCleanUpInterval() {
-		return cleanUpInterval;
-	}
-
-	/**
-	 * Returns back pressure statistics for a operator.
-	 *
-	 * @param vertex Operator to get the stats for.
-	 *
-	 * @return Back pressure statistics for an operator
-	 */
-	public Option<OperatorBackPressureStats> getOperatorBackPressureStats(ExecutionJobVertex vertex) {
-		return Option.apply(operatorStatsCache.getIfPresent(vertex));
-	}
-
-	/**
-	 * Triggers a stack trace sample for a operator to gather the back pressure
-	 * statistics. If there is a sample in progress for the operator, the call
-	 * is ignored.
-	 *
-	 * @param vertex Operator to get the stats for.
-	 * @return Flag indicating whether a sample with triggered.
-	 */
-	@SuppressWarnings("unchecked")
-	public boolean triggerStackTraceSample(ExecutionJobVertex vertex) {
-		synchronized (lock) {
-			if (shutDown) {
-				return false;
-			}
-
-			if (!pendingStats.contains(vertex) &&
-					!vertex.getGraph().getState().isGloballyTerminalState()) {
-
-				Executor executor = vertex.getGraph().getFutureExecutor();
-
-				// Only trigger if still active job
-				if (executor != null) {
-					pendingStats.add(vertex);
-
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Triggering stack trace sample for tasks: " + Arrays.toString(vertex.getTaskVertices()));
-					}
-
-					CompletableFuture<StackTraceSample> sample = coordinator.triggerStackTraceSample(
-							vertex.getTaskVertices(),
-							numSamples,
-							delayBetweenSamples,
-							MAX_STACK_TRACE_DEPTH);
-
-					sample.handleAsync(new StackTraceSampleCompletionCallback(vertex), executor);
-
-					return true;
-				}
-			}
-
-			return false;
-		}
-	}
-
-	/**
-	 * Cleans up the operator stats cache if it contains timed out entries.
-	 *
-	 * <p>The Guava cache only evicts as maintenance during normal operations.
-	 * If this handler is inactive, it will never be cleaned.
-	 */
-	public void cleanUpOperatorStatsCache() {
-		operatorStatsCache.cleanUp();
-	}
-
-	/**
-	 * Shuts down the stats tracker.
-	 *
-	 * <p>Invalidates the cache and clears all pending stats.
-	 */
-	public void shutDown() {
-		synchronized (lock) {
-			if (!shutDown) {
-				operatorStatsCache.invalidateAll();
-				pendingStats.clear();
-
-				shutDown = true;
-			}
-		}
-	}
-
-	/**
-	 * Invalidates the cache (irrespective of clean up interval).
-	 */
-	void invalidateOperatorStatsCache() {
-		operatorStatsCache.invalidateAll();
-	}
-
-	/**
-	 * Callback on completed stack trace sample.
-	 */
-	class StackTraceSampleCompletionCallback implements BiFunction<StackTraceSample, Throwable, Void> {
-
-		private final ExecutionJobVertex vertex;
-
-		public StackTraceSampleCompletionCallback(ExecutionJobVertex vertex) {
-			this.vertex = vertex;
-		}
-
-		@Override
-		public Void apply(StackTraceSample stackTraceSample, Throwable throwable) {
-			synchronized (lock) {
-				try {
-					if (shutDown) {
-						return null;
-					}
-
-					// Job finished, ignore.
-					JobStatus jobState = vertex.getGraph().getState();
-					if (jobState.isGloballyTerminalState()) {
-						LOG.debug("Ignoring sample, because job is in state " + jobState + ".");
-					} else if (stackTraceSample != null) {
-						OperatorBackPressureStats stats = createStatsFromSample(stackTraceSample);
-						operatorStatsCache.put(vertex, stats);
-					} else {
-						LOG.debug("Failed to gather stack trace sample.", throwable);
-					}
-				} catch (Throwable t) {
-					LOG.error("Error during stats completion.", t);
-				} finally {
-					pendingStats.remove(vertex);
-				}
-
-				return null;
-			}
-		}
-
-		/**
-		 * Creates the back pressure stats from a stack trace sample.
-		 *
-		 * @param sample Stack trace sample to base stats on.
-		 *
-		 * @return Back pressure stats
-		 */
-		private OperatorBackPressureStats createStatsFromSample(StackTraceSample sample) {
-			Map<ExecutionAttemptID, List<StackTraceElement[]>> traces = sample.getStackTraces();
-
-			// Map task ID to subtask index, because the web interface expects
-			// it like that.
-			Map<ExecutionAttemptID, Integer> subtaskIndexMap = Maps
-					.newHashMapWithExpectedSize(traces.size());
-
-			Set<ExecutionAttemptID> sampledTasks = sample.getStackTraces().keySet();
-
-			for (ExecutionVertex task : vertex.getTaskVertices()) {
-				ExecutionAttemptID taskId = task.getCurrentExecutionAttempt().getAttemptId();
-				if (sampledTasks.contains(taskId)) {
-					subtaskIndexMap.put(taskId, task.getParallelSubtaskIndex());
-				} else {
-					LOG.debug("Outdated sample. A task, which is part of the " +
-							"sample has been reset.");
-				}
-			}
-
-			// Ratio of blocked samples to total samples per sub task. Array
-			// position corresponds to sub task index.
-			double[] backPressureRatio = new double[traces.size()];
-
-			for (Entry<ExecutionAttemptID, List<StackTraceElement[]>> entry : traces.entrySet()) {
-				int backPressureSamples = 0;
-
-				List<StackTraceElement[]> taskTraces = entry.getValue();
-
-				for (StackTraceElement[] trace : taskTraces) {
-					for (int i = trace.length - 1; i >= 0; i--) {
-						StackTraceElement elem = trace[i];
-
-						if (elem.getClassName().equals(EXPECTED_CLASS_NAME) &&
-								elem.getMethodName().equals(EXPECTED_METHOD_NAME)) {
-
-							backPressureSamples++;
-							break; // Continue with next stack trace
-						}
-					}
-				}
-
-				int subtaskIndex = subtaskIndexMap.get(entry.getKey());
-
-				int size = taskTraces.size();
-				double ratio = (size > 0)
-						? ((double) backPressureSamples) / size
-						: 0;
-
-				backPressureRatio[subtaskIndex] = ratio;
-			}
-
-			return new OperatorBackPressureStats(
-					sample.getSampleId(),
-					sample.getEndTime(),
-					backPressureRatio);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
deleted file mode 100644
index 8a96969..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
+++ /dev/null
@@ -1,86 +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.webmonitor;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Optional;
-import java.util.WeakHashMap;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
- *
- * <p>The holder will cache the ExecutionGraph behind a weak reference, which will be cleared
- * at some point once no one else is pointing to the ExecutionGraph.
- * Note that while the holder runs in the same JVM as the JobManager or Archive, the reference should
- * stay valid.
- */
-public class ExecutionGraphHolder {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraphHolder.class);
-
-	private final Time timeout;
-
-	private final WeakHashMap<JobID, AccessExecutionGraph> cache = new WeakHashMap<>();
-
-	public ExecutionGraphHolder() {
-		this(WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
-	}
-
-	public ExecutionGraphHolder(Time timeout) {
-		this.timeout = checkNotNull(timeout);
-	}
-
-	/**
-	 * Retrieves the execution graph with {@link JobID} jid wrapped in {@link Optional} or
-	 * {@link Optional#empty()} if it cannot be found.
-	 *
-	 * @param jid jobID of the execution graph to be retrieved
-	 * @return Optional ExecutionGraph if it has been retrievable, empty if there has been no ExecutionGraph
-	 */
-	public CompletableFuture<Optional<AccessExecutionGraph>> getExecutionGraph(JobID jid, JobManagerGateway jobManagerGateway) {
-		AccessExecutionGraph cached = cache.get(jid);
-		if (cached != null) {
-			if (cached.getState() == JobStatus.SUSPENDED) {
-				cache.remove(jid);
-			} else {
-				return CompletableFuture.completedFuture(Optional.of(cached));
-			}
-		}
-
-		CompletableFuture<Optional<AccessExecutionGraph>> executionGraphFuture = jobManagerGateway.requestJob(jid, timeout);
-
-		executionGraphFuture.thenAcceptAsync(
-			optExecutionGraph ->
-				optExecutionGraph.ifPresent(executionGraph -> cache.put(jid, executionGraph)));
-
-		return executionGraphFuture;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/NotFoundException.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/NotFoundException.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/NotFoundException.java
deleted file mode 100644
index 71125c9..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/NotFoundException.java
+++ /dev/null
@@ -1,32 +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.webmonitor;
-
-/**
- * A special exception that indicates that an element was not found and that the
- * request should be answered with a {@code 404} return code.
- */
-public class NotFoundException extends Exception {
-
-	private static final long serialVersionUID = -4036006746423754639L;
-
-	public NotFoundException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/OperatorBackPressureStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/OperatorBackPressureStats.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/OperatorBackPressureStats.java
deleted file mode 100644
index bfd5be2..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/OperatorBackPressureStats.java
+++ /dev/null
@@ -1,126 +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.webmonitor;
-
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Back pressure statistics of multiple tasks.
- *
- * <p>Statistics are gathered by sampling stack traces of running tasks. The
- * back pressure ratio denotes the ratio of traces indicating back pressure
- * to the total number of sampled traces.
- */
-public class OperatorBackPressureStats {
-
-	/** ID of the corresponding sample. */
-	private final int sampleId;
-
-	/** End time stamp of the corresponding sample. */
-	private final long endTimestamp;
-
-	/** Back pressure ratio per subtask. */
-	private final double[] subTaskBackPressureRatio;
-
-	/** Maximum back pressure ratio. */
-	private final double maxSubTaskBackPressureRatio;
-
-	public OperatorBackPressureStats(
-			int sampleId,
-			long endTimestamp,
-			double[] subTaskBackPressureRatio) {
-
-		this.sampleId = sampleId;
-		this.endTimestamp = endTimestamp;
-		this.subTaskBackPressureRatio = checkNotNull(subTaskBackPressureRatio, "Sub task back pressure ratio");
-		checkArgument(subTaskBackPressureRatio.length >= 1, "No Sub task back pressure ratio specified");
-
-		double max = 0;
-		for (double ratio : subTaskBackPressureRatio) {
-			if (ratio > max) {
-				max = ratio;
-			}
-		}
-
-		maxSubTaskBackPressureRatio = max;
-	}
-
-	/**
-	 * Returns the ID of the sample.
-	 *
-	 * @return ID of the sample
-	 */
-	public int getSampleId() {
-		return sampleId;
-	}
-
-	/**
-	 * Returns the time stamp, when all stack traces were collected at the
-	 * JobManager.
-	 *
-	 * @return Time stamp, when all stack traces were collected at the
-	 * JobManager
-	 */
-	public long getEndTimestamp() {
-		return endTimestamp;
-	}
-
-	/**
-	 * Returns the number of sub tasks.
-	 *
-	 * @return Number of sub tasks.
-	 */
-	public int getNumberOfSubTasks() {
-		return subTaskBackPressureRatio.length;
-	}
-
-	/**
-	 * Returns the ratio of stack traces indicating back pressure to total
-	 * number of sampled stack traces.
-	 *
-	 * @param index Subtask index.
-	 *
-	 * @return Ratio of stack traces indicating back pressure to total number
-	 * of sampled stack traces.
-	 */
-	public double getBackPressureRatio(int index) {
-		return subTaskBackPressureRatio[index];
-	}
-
-	/**
-	 * Returns the maximum back pressure ratio of all sub tasks.
-	 *
-	 * @return Maximum back pressure ratio of all sub tasks.
-	 */
-	public double getMaxBackPressureRatio() {
-		return maxSubTaskBackPressureRatio;
-	}
-
-	@Override
-	public String toString() {
-		return "OperatorBackPressureStats{" +
-				"sampleId=" + sampleId +
-				", endTimestamp=" + endTimestamp +
-				", subTaskBackPressureRatio=" + Arrays.toString(subTaskBackPressureRatio) +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index b393021..993a225 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -21,9 +21,11 @@ package org.apache.flink.runtime.webmonitor;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.NotFoundException;
 import org.apache.flink.runtime.rest.handler.RedirectHandler;
+import org.apache.flink.runtime.rest.handler.WebHandler;
+import org.apache.flink.runtime.rest.handler.legacy.RequestHandler;
 import org.apache.flink.runtime.rest.handler.util.HandlerRedirectUtils;
-import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
 import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
 import org.apache.flink.util.ExceptionUtils;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSample.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSample.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSample.java
deleted file mode 100644
index d60f8a4..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSample.java
+++ /dev/null
@@ -1,119 +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.webmonitor;
-
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/**
- * A sample of stack traces for one or more tasks.
- *
- * <p>The sampling is triggered in {@link StackTraceSampleCoordinator}.
- */
-public class StackTraceSample {
-
-	/** ID of this sample (unique per job). */
-	private final int sampleId;
-
-	/** Time stamp, when the sample was triggered. */
-	private final long startTime;
-
-	/** Time stamp, when all stack traces were collected at the JobManager. */
-	private final long endTime;
-
-	/** Map of stack traces by execution ID. */
-	private final Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask;
-
-	/**
-	 * Creates a stack trace sample.
-	 *
-	 * @param sampleId          ID of the sample.
-	 * @param startTime         Time stamp, when the sample was triggered.
-	 * @param endTime           Time stamp, when all stack traces were
-	 *                          collected at the JobManager.
-	 * @param stackTracesByTask Map of stack traces by execution ID.
-	 */
-	public StackTraceSample(
-			int sampleId,
-			long startTime,
-			long endTime,
-			Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask) {
-
-		checkArgument(sampleId >= 0, "Negative sample ID");
-		checkArgument(startTime >= 0, "Negative start time");
-		checkArgument(endTime >= startTime, "End time before start time");
-
-		this.sampleId = sampleId;
-		this.startTime = startTime;
-		this.endTime = endTime;
-		this.stackTracesByTask = Collections.unmodifiableMap(stackTracesByTask);
-	}
-
-	/**
-	 * Returns the ID of the sample.
-	 *
-	 * @return ID of the sample
-	 */
-	public int getSampleId() {
-		return sampleId;
-	}
-
-	/**
-	 * Returns the time stamp, when the sample was triggered.
-	 *
-	 * @return Time stamp, when the sample was triggered
-	 */
-	public long getStartTime() {
-		return startTime;
-	}
-
-	/**
-	 * Returns the time stamp, when all stack traces were collected at the
-	 * JobManager.
-	 *
-	 * @return Time stamp, when all stack traces were collected at the
-	 * JobManager
-	 */
-	public long getEndTime() {
-		return endTime;
-	}
-
-	/**
-	 * Returns the a map of stack traces by execution ID.
-	 *
-	 * @return Map of stack traces by execution ID
-	 */
-	public Map<ExecutionAttemptID, List<StackTraceElement[]>> getStackTraces() {
-		return stackTracesByTask;
-	}
-
-	@Override
-	public String toString() {
-		return "StackTraceSample{" +
-				"sampleId=" + sampleId +
-				", startTime=" + startTime +
-				", endTime=" + endTime +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinator.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinator.java
deleted file mode 100644
index 534d2fa..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinator.java
+++ /dev/null
@@ -1,392 +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.webmonitor;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.messages.StackTraceSampleResponse;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayDeque;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A coordinator for triggering and collecting stack traces of running tasks.
- */
-public class StackTraceSampleCoordinator {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StackTraceSampleCoordinator.class);
-
-	private static final int NUM_GHOST_SAMPLE_IDS = 10;
-
-	private final Object lock = new Object();
-
-	/** Executor used to run the futures. */
-	private final Executor executor;
-
-	/** Time out after the expected sampling duration. */
-	private final long sampleTimeout;
-
-	/** In progress samples (guarded by lock). */
-	private final Map<Integer, PendingStackTraceSample> pendingSamples = new HashMap<>();
-
-	/** A list of recent sample IDs to identify late messages vs. invalid ones. */
-	private final ArrayDeque<Integer> recentPendingSamples = new ArrayDeque<>(NUM_GHOST_SAMPLE_IDS);
-
-	/** Sample ID counter (guarded by lock). */
-	private int sampleIdCounter;
-
-	/**
-	 * Flag indicating whether the coordinator is still running (guarded by
-	 * lock).
-	 */
-	private boolean isShutDown;
-
-	/**
-	 * Creates a new coordinator for the job.
-	 *
-	 * @param executor to use to execute the futures
-	 * @param sampleTimeout Time out after the expected sampling duration.
-	 *                      This is added to the expected duration of a
-	 *                      sample, which is determined by the number of
-	 *                      samples and the delay between each sample.
-	 */
-	public StackTraceSampleCoordinator(Executor executor, long sampleTimeout) {
-		checkArgument(sampleTimeout >= 0L);
-		this.executor = Preconditions.checkNotNull(executor);
-		this.sampleTimeout = sampleTimeout;
-	}
-
-	/**
-	 * Triggers a stack trace sample to all tasks.
-	 *
-	 * @param tasksToSample       Tasks to sample.
-	 * @param numSamples          Number of stack trace samples to collect.
-	 * @param delayBetweenSamples Delay between consecutive samples.
-	 * @param maxStackTraceDepth  Maximum depth of the stack trace. 0 indicates
-	 *                            no maximum and keeps the complete stack trace.
-	 * @return A future of the completed stack trace sample
-	 */
-	@SuppressWarnings("unchecked")
-	public CompletableFuture<StackTraceSample> triggerStackTraceSample(
-			ExecutionVertex[] tasksToSample,
-			int numSamples,
-			Time delayBetweenSamples,
-			int maxStackTraceDepth) {
-
-		checkNotNull(tasksToSample, "Tasks to sample");
-		checkArgument(tasksToSample.length >= 1, "No tasks to sample");
-		checkArgument(numSamples >= 1, "No number of samples");
-		checkArgument(maxStackTraceDepth >= 0, "Negative maximum stack trace depth");
-
-		// Execution IDs of running tasks
-		ExecutionAttemptID[] triggerIds = new ExecutionAttemptID[tasksToSample.length];
-		Execution[] executions = new Execution[tasksToSample.length];
-
-		// Check that all tasks are RUNNING before triggering anything. The
-		// triggering can still fail.
-		for (int i = 0; i < triggerIds.length; i++) {
-			Execution execution = tasksToSample[i].getCurrentExecutionAttempt();
-			if (execution != null && execution.getState() == ExecutionState.RUNNING) {
-				executions[i] = execution;
-				triggerIds[i] = execution.getAttemptId();
-			} else {
-				return FutureUtils.completedExceptionally(new IllegalStateException("Task " + tasksToSample[i]
-					.getTaskNameWithSubtaskIndex() + " is not running."));
-			}
-		}
-
-		synchronized (lock) {
-			if (isShutDown) {
-				return FutureUtils.completedExceptionally(new IllegalStateException("Shut down"));
-			}
-
-			final int sampleId = sampleIdCounter++;
-
-			LOG.debug("Triggering stack trace sample {}", sampleId);
-
-			final PendingStackTraceSample pending = new PendingStackTraceSample(
-					sampleId, triggerIds);
-
-			// Discard the sample if it takes too long. We don't send cancel
-			// messages to the task managers, but only wait for the responses
-			// and then ignore them.
-			long expectedDuration = numSamples * delayBetweenSamples.toMilliseconds();
-			Time timeout = Time.milliseconds(expectedDuration + sampleTimeout);
-
-			// Add the pending sample before scheduling the discard task to
-			// prevent races with removing it again.
-			pendingSamples.put(sampleId, pending);
-
-			// Trigger all samples
-			for (Execution execution: executions) {
-				final CompletableFuture<StackTraceSampleResponse> stackTraceSampleFuture = execution.requestStackTraceSample(
-					sampleId,
-					numSamples,
-					delayBetweenSamples,
-					maxStackTraceDepth,
-					timeout);
-
-				stackTraceSampleFuture.handleAsync(
-					(StackTraceSampleResponse stackTraceSampleResponse, Throwable throwable) -> {
-						if (stackTraceSampleResponse != null) {
-							collectStackTraces(
-								stackTraceSampleResponse.getSampleId(),
-								stackTraceSampleResponse.getExecutionAttemptID(),
-								stackTraceSampleResponse.getSamples());
-						} else {
-							cancelStackTraceSample(sampleId, throwable);
-						}
-
-						return null;
-					},
-					executor);
-			}
-
-			return pending.getStackTraceSampleFuture();
-		}
-	}
-
-	/**
-	 * Cancels a pending sample.
-	 *
-	 * @param sampleId ID of the sample to cancel.
-	 * @param cause Cause of the cancelling (can be <code>null</code>).
-	 */
-	public void cancelStackTraceSample(int sampleId, Throwable cause) {
-		synchronized (lock) {
-			if (isShutDown) {
-				return;
-			}
-
-			PendingStackTraceSample sample = pendingSamples.remove(sampleId);
-			if (sample != null) {
-				if (cause != null) {
-					LOG.info("Cancelling sample " + sampleId, cause);
-				} else {
-					LOG.info("Cancelling sample {}", sampleId);
-				}
-
-				sample.discard(cause);
-				rememberRecentSampleId(sampleId);
-			}
-		}
-	}
-
-	/**
-	 * Shuts down the coordinator.
-	 *
-	 * <p>After shut down, no further operations are executed.
-	 */
-	public void shutDown() {
-		synchronized (lock) {
-			if (!isShutDown) {
-				LOG.info("Shutting down stack trace sample coordinator.");
-
-				for (PendingStackTraceSample pending : pendingSamples.values()) {
-					pending.discard(new RuntimeException("Shut down"));
-				}
-
-				pendingSamples.clear();
-
-				isShutDown = true;
-			}
-		}
-	}
-
-	/**
-	 * Collects stack traces of a task.
-	 *
-	 * @param sampleId    ID of the sample.
-	 * @param executionId ID of the sampled task.
-	 * @param stackTraces Stack traces of the sampled task.
-	 *
-	 * @throws IllegalStateException If unknown sample ID and not recently
-	 *                               finished or cancelled sample.
-	 */
-	public void collectStackTraces(
-			int sampleId,
-			ExecutionAttemptID executionId,
-			List<StackTraceElement[]> stackTraces) {
-
-		synchronized (lock) {
-			if (isShutDown) {
-				return;
-			}
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Collecting stack trace sample {} of task {}", sampleId, executionId);
-			}
-
-			PendingStackTraceSample pending = pendingSamples.get(sampleId);
-
-			if (pending != null) {
-				pending.collectStackTraces(executionId, stackTraces);
-
-				// Publish the sample
-				if (pending.isComplete()) {
-					pendingSamples.remove(sampleId);
-					rememberRecentSampleId(sampleId);
-
-					pending.completePromiseAndDiscard();
-				}
-			} else if (recentPendingSamples.contains(sampleId)) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Received late stack trace sample {} of task {}",
-							sampleId, executionId);
-				}
-			} else {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Unknown sample ID " + sampleId);
-				}
-			}
-		}
-	}
-
-	private void rememberRecentSampleId(int sampleId) {
-		if (recentPendingSamples.size() >= NUM_GHOST_SAMPLE_IDS) {
-			recentPendingSamples.removeFirst();
-		}
-		recentPendingSamples.addLast(sampleId);
-	}
-
-	int getNumberOfPendingSamples() {
-		synchronized (lock) {
-			return pendingSamples.size();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A pending stack trace sample, which collects stack traces and owns a
-	 * {@link StackTraceSample} promise.
-	 *
-	 * <p>Access pending sample in lock scope.
-	 */
-	private static class PendingStackTraceSample {
-
-		private final int sampleId;
-		private final long startTime;
-		private final Set<ExecutionAttemptID> pendingTasks;
-		private final Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask;
-		private final CompletableFuture<StackTraceSample> stackTraceFuture;
-
-		private boolean isDiscarded;
-
-		PendingStackTraceSample(
-				int sampleId,
-				ExecutionAttemptID[] tasksToCollect) {
-
-			this.sampleId = sampleId;
-			this.startTime = System.currentTimeMillis();
-			this.pendingTasks = new HashSet<>(Arrays.asList(tasksToCollect));
-			this.stackTracesByTask = Maps.newHashMapWithExpectedSize(tasksToCollect.length);
-			this.stackTraceFuture = new CompletableFuture<>();
-		}
-
-		int getSampleId() {
-			return sampleId;
-		}
-
-		long getStartTime() {
-			return startTime;
-		}
-
-		boolean isDiscarded() {
-			return isDiscarded;
-		}
-
-		boolean isComplete() {
-			if (isDiscarded) {
-				throw new IllegalStateException("Discarded");
-			}
-
-			return pendingTasks.isEmpty();
-		}
-
-		void discard(Throwable cause) {
-			if (!isDiscarded) {
-				pendingTasks.clear();
-				stackTracesByTask.clear();
-
-				stackTraceFuture.completeExceptionally(new RuntimeException("Discarded", cause));
-
-				isDiscarded = true;
-			}
-		}
-
-		void collectStackTraces(ExecutionAttemptID executionId, List<StackTraceElement[]> stackTraces) {
-			if (isDiscarded) {
-				throw new IllegalStateException("Discarded");
-			}
-
-			if (pendingTasks.remove(executionId)) {
-				stackTracesByTask.put(executionId, Collections.unmodifiableList(stackTraces));
-			} else if (isComplete()) {
-				throw new IllegalStateException("Completed");
-			} else {
-				throw new IllegalArgumentException("Unknown task " + executionId);
-			}
-		}
-
-		void completePromiseAndDiscard() {
-			if (isComplete()) {
-				isDiscarded = true;
-
-				long endTime = System.currentTimeMillis();
-
-				StackTraceSample stackTraceSample = new StackTraceSample(
-						sampleId,
-						startTime,
-						endTime,
-						stackTracesByTask);
-
-				stackTraceFuture.complete(stackTraceSample);
-			} else {
-				throw new IllegalStateException("Not completed yet");
-			}
-		}
-
-		@SuppressWarnings("unchecked")
-		CompletableFuture<StackTraceSample> getStackTraceSampleFuture() {
-			return stackTraceFuture;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebHandler.java
deleted file mode 100644
index 9839abd..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebHandler.java
+++ /dev/null
@@ -1,32 +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.webmonitor;
-
-/**
- * Marker interface for web handlers which can describe their paths.
- */
-public interface WebHandler {
-
-	/**
-	 * Returns an array of REST URL's under which this handler can be registered.
-	 *
-	 * @return array containing REST URL's under which this handler can be registered.
-	 */
-	String[] getPaths();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 71e1593..cd128de 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -27,50 +27,54 @@ import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.net.SSLUtils;
-import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
-import org.apache.flink.runtime.webmonitor.handlers.ClusterOverviewHandler;
-import org.apache.flink.runtime.webmonitor.handlers.ConstantTextHandler;
-import org.apache.flink.runtime.webmonitor.handlers.CurrentJobIdsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
-import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
+import org.apache.flink.runtime.rest.handler.WebHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ClusterOverviewHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ConstantTextHandler;
+import org.apache.flink.runtime.rest.handler.legacy.CurrentJobIdsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.CurrentJobsOverviewHandler;
+import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.rest.handler.legacy.JobAccumulatorsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobCancellationHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobCancellationWithSavepointHandlers;
+import org.apache.flink.runtime.rest.handler.legacy.JobConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobExceptionsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobManagerConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobPlanHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobStoppingHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobVertexAccumulatorsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobVertexBackPressureHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobVertexDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JobVertexTaskManagersHandler;
+import org.apache.flink.runtime.rest.handler.legacy.RequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.SubtaskCurrentAttemptDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.SubtaskExecutionAttemptAccumulatorsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.SubtaskExecutionAttemptDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.SubtasksAllAccumulatorsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.SubtasksTimesHandler;
+import org.apache.flink.runtime.rest.handler.legacy.TaskManagerLogHandler;
+import org.apache.flink.runtime.rest.handler.legacy.TaskManagersHandler;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.StackTraceSampleCoordinator;
+import org.apache.flink.runtime.rest.handler.legacy.checkpoints.CheckpointConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.checkpoints.CheckpointStatsCache;
+import org.apache.flink.runtime.rest.handler.legacy.checkpoints.CheckpointStatsDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.checkpoints.CheckpointStatsDetailsSubtasksHandler;
+import org.apache.flink.runtime.rest.handler.legacy.checkpoints.CheckpointStatsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.JobManagerMetricsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.JobMetricsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.JobVertexMetricsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.TaskManagerMetricsHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarAccessDeniedHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarDeleteHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarListHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarPlanHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarRunHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JarUploadHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobAccumulatorsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobCancellationHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobCancellationWithSavepointHandlers;
-import org.apache.flink.runtime.webmonitor.handlers.JobConfigHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobDetailsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobExceptionsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobManagerConfigHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobPlanHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobStoppingHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobVertexAccumulatorsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobVertexBackPressureHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobVertexDetailsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JobVertexTaskManagersHandler;
-import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.SubtaskCurrentAttemptDetailsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.SubtaskExecutionAttemptAccumulatorsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.SubtaskExecutionAttemptDetailsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.SubtasksAllAccumulatorsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.SubtasksTimesHandler;
-import org.apache.flink.runtime.webmonitor.handlers.TaskManagerLogHandler;
-import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
-import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointConfigHandler;
-import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsCache;
-import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsDetailsHandler;
-import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsDetailsSubtasksHandler;
-import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsHandler;
 import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.metrics.JobManagerMetricsHandler;
-import org.apache.flink.runtime.webmonitor.metrics.JobMetricsHandler;
-import org.apache.flink.runtime.webmonitor.metrics.JobVertexMetricsHandler;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.metrics.TaskManagerMetricsHandler;
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
 import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
@@ -189,7 +193,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			this.uploadDir = null;
 		}
 
-		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder();
+		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder(timeout);
 
 		// - Back pressure stats ----------------------------------------------
 
@@ -255,7 +259,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		get(router, new SubtasksTimesHandler(currentGraphs, executor));
 		get(router, new JobVertexTaskManagersHandler(currentGraphs, executor, metricFetcher));
 		get(router, new JobVertexAccumulatorsHandler(currentGraphs, executor));
-		get(router, new JobVertexBackPressureHandler(currentGraphs, executor,	backPressureStatsTracker, refreshInterval));
+		get(router, new JobVertexBackPressureHandler(currentGraphs, executor, backPressureStatsTracker, refreshInterval));
 		get(router, new JobVertexMetricsHandler(executor, metricFetcher));
 		get(router, new SubtasksAllAccumulatorsHandler(currentGraphs, executor));
 		get(router, new SubtaskCurrentAttemptDetailsHandler(currentGraphs, executor, metricFetcher));

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
deleted file mode 100644
index 2445d3f..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ /dev/null
@@ -1,363 +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.webmonitor.files;
-
-/*****************************************************************************
- * This code is based on the "HttpStaticFileServerHandler" from the
- * Netty project's HTTP server example.
- *
- * See http://netty.io and
- * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
- *****************************************************************************/
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.rest.handler.RedirectHandler;
-import org.apache.flink.runtime.rest.handler.util.MimeTypes;
-import org.apache.flink.runtime.webmonitor.RestfulGateway;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
-import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
-import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.RandomAccessFile;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.Locale;
-import java.util.TimeZone;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.DATE;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.EXPIRES;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.LAST_MODIFIED;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Simple file server handler that serves requests to web frontend's static files, such as
- * HTML, CSS, or JS files.
- *
- * <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
- * example.</p>
- */
-@ChannelHandler.Sharable
-public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectHandler<T> {
-
-	/** Timezone in which this server answers its "if-modified" requests. */
-	private static final TimeZone GMT_TIMEZONE = TimeZone.getTimeZone("GMT");
-
-	/** Date format for HTTP. */
-	public static final String HTTP_DATE_FORMAT = "EEE, dd MMM yyyy HH:mm:ss zzz";
-
-	/** Be default, we allow files to be cached for 5 minutes. */
-	private static final int HTTP_CACHE_SECONDS = 300;
-
-	// ------------------------------------------------------------------------
-
-	/** The path in which the static documents are. */
-	private final File rootPath;
-
-	public StaticFileServerHandler(
-			GatewayRetriever<T> retriever,
-			CompletableFuture<String> localJobManagerAddressFuture,
-			Time timeout,
-			File rootPath) throws IOException {
-
-		super(localJobManagerAddressFuture, retriever, timeout);
-
-		this.rootPath = checkNotNull(rootPath).getCanonicalFile();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Responses to requests
-	// ------------------------------------------------------------------------
-
-	@Override
-	protected void respondAsLeader(ChannelHandlerContext channelHandlerContext, Routed routed, T gateway) throws Exception {
-		final HttpRequest request = routed.request();
-		final String requestPath;
-
-		// make sure we request the "index.html" in case there is a directory request
-		if (routed.path().endsWith("/")) {
-			requestPath = routed.path() + "index.html";
-		}
-		// in case the files being accessed are logs or stdout files, find appropriate paths.
-		else if (routed.path().equals("/jobmanager/log") || routed.path().equals("/jobmanager/stdout")) {
-			requestPath = "";
-		} else {
-			requestPath = routed.path();
-		}
-
-		respondToRequest(channelHandlerContext, request, requestPath);
-	}
-
-	/**
-	 * Response when running with leading JobManager.
-	 */
-	private void respondToRequest(ChannelHandlerContext ctx, HttpRequest request, String requestPath)
-			throws IOException, ParseException, URISyntaxException {
-
-		// convert to absolute path
-		final File file = new File(rootPath, requestPath);
-
-		if (!file.exists()) {
-			// file does not exist. Try to load it with the classloader
-			ClassLoader cl = StaticFileServerHandler.class.getClassLoader();
-
-			try (InputStream resourceStream = cl.getResourceAsStream("web" + requestPath)) {
-				boolean success = false;
-				try {
-					if (resourceStream != null) {
-						URL root = cl.getResource("web");
-						URL requested = cl.getResource("web" + requestPath);
-
-						if (root != null && requested != null) {
-							URI rootURI = new URI(root.getPath()).normalize();
-							URI requestedURI = new URI(requested.getPath()).normalize();
-
-							// Check that we don't load anything from outside of the
-							// expected scope.
-							if (!rootURI.relativize(requestedURI).equals(requestedURI)) {
-								logger.debug("Loading missing file from classloader: {}", requestPath);
-								// ensure that directory to file exists.
-								file.getParentFile().mkdirs();
-								Files.copy(resourceStream, file.toPath());
-
-								success = true;
-							}
-						}
-					}
-				} catch (Throwable t) {
-					logger.error("error while responding", t);
-				} finally {
-					if (!success) {
-						logger.debug("Unable to load requested file {} from classloader", requestPath);
-						sendError(ctx, NOT_FOUND);
-						return;
-					}
-				}
-			}
-		}
-
-		if (!file.exists() || file.isHidden() || file.isDirectory() || !file.isFile()) {
-			sendError(ctx, NOT_FOUND);
-			return;
-		}
-
-		if (!file.getCanonicalFile().toPath().startsWith(rootPath.toPath())) {
-			sendError(ctx, NOT_FOUND);
-			return;
-		}
-
-		// cache validation
-		final String ifModifiedSince = request.headers().get(IF_MODIFIED_SINCE);
-		if (ifModifiedSince != null && !ifModifiedSince.isEmpty()) {
-			SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
-			Date ifModifiedSinceDate = dateFormatter.parse(ifModifiedSince);
-
-			// Only compare up to the second because the datetime format we send to the client
-			// does not have milliseconds
-			long ifModifiedSinceDateSeconds = ifModifiedSinceDate.getTime() / 1000;
-			long fileLastModifiedSeconds = file.lastModified() / 1000;
-			if (ifModifiedSinceDateSeconds == fileLastModifiedSeconds) {
-				if (logger.isDebugEnabled()) {
-					logger.debug("Responding 'NOT MODIFIED' for file '" + file.getAbsolutePath() + '\'');
-				}
-
-				sendNotModified(ctx);
-				return;
-			}
-		}
-
-		if (logger.isDebugEnabled()) {
-			logger.debug("Responding with file '" + file.getAbsolutePath() + '\'');
-		}
-
-		// Don't need to close this manually. Netty's DefaultFileRegion will take care of it.
-		final RandomAccessFile raf;
-		try {
-			raf = new RandomAccessFile(file, "r");
-		}
-		catch (FileNotFoundException e) {
-			sendError(ctx, NOT_FOUND);
-			return;
-		}
-
-		try {
-			long fileLength = raf.length();
-
-			HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-			setContentTypeHeader(response, file);
-
-			// since the log and out files are rapidly changing, we don't want to browser to cache them
-			if (!(requestPath.contains("log") || requestPath.contains("out"))) {
-				setDateAndCacheHeaders(response, file);
-			}
-			if (HttpHeaders.isKeepAlive(request)) {
-				response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
-			}
-			HttpHeaders.setContentLength(response, fileLength);
-
-			// write the initial line and the header.
-			ctx.write(response);
-
-			// write the content.
-			ChannelFuture lastContentFuture;
-			if (ctx.pipeline().get(SslHandler.class) == null) {
-				ctx.write(new DefaultFileRegion(raf.getChannel(), 0, fileLength), ctx.newProgressivePromise());
-				lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
-			} else {
-				lastContentFuture = ctx.writeAndFlush(new HttpChunkedInput(new ChunkedFile(raf, 0, fileLength, 8192)),
-					ctx.newProgressivePromise());
-				// HttpChunkedInput will write the end marker (LastHttpContent) for us.
-			}
-
-			// close the connection, if no keep-alive is needed
-			if (!HttpHeaders.isKeepAlive(request)) {
-				lastContentFuture.addListener(ChannelFutureListener.CLOSE);
-			}
-		} catch (Exception e) {
-			raf.close();
-			logger.error("Failed to serve file.", e);
-			sendError(ctx, INTERNAL_SERVER_ERROR);
-		}
-	}
-
-	@Override
-	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-		if (ctx.channel().isActive()) {
-			logger.error("Caught exception", cause);
-			sendError(ctx, INTERNAL_SERVER_ERROR);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities to encode headers and responses
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Writes a simple  error response message.
-	 *
-	 * @param ctx    The channel context to write the response to.
-	 * @param status The response status.
-	 */
-	public static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
-		FullHttpResponse response = new DefaultFullHttpResponse(
-				HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8));
-		response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8");
-
-		// close the connection as soon as the error message is sent.
-		ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
-	}
-
-	/**
-	 * Send the "304 Not Modified" response. This response can be used when the
-	 * file timestamp is the same as what the browser is sending up.
-	 *
-	 * @param ctx The channel context to write the response to.
-	 */
-	public static void sendNotModified(ChannelHandlerContext ctx) {
-		FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, NOT_MODIFIED);
-		setDateHeader(response);
-
-		// close the connection as soon as the error message is sent.
-		ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
-	}
-
-	/**
-	 * Sets the "date" header for the HTTP response.
-	 *
-	 * @param response HTTP response
-	 */
-	public static void setDateHeader(FullHttpResponse response) {
-		SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
-		dateFormatter.setTimeZone(GMT_TIMEZONE);
-
-		Calendar time = new GregorianCalendar();
-		response.headers().set(DATE, dateFormatter.format(time.getTime()));
-	}
-
-	/**
-	 * Sets the "date" and "cache" headers for the HTTP Response.
-	 *
-	 * @param response    The HTTP response object.
-	 * @param fileToCache File to extract the modification timestamp from.
-	 */
-	public static void setDateAndCacheHeaders(HttpResponse response, File fileToCache) {
-		SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
-		dateFormatter.setTimeZone(GMT_TIMEZONE);
-
-		// date header
-		Calendar time = new GregorianCalendar();
-		response.headers().set(DATE, dateFormatter.format(time.getTime()));
-
-		// cache headers
-		time.add(Calendar.SECOND, HTTP_CACHE_SECONDS);
-		response.headers().set(EXPIRES, dateFormatter.format(time.getTime()));
-		response.headers().set(CACHE_CONTROL, "private, max-age=" + HTTP_CACHE_SECONDS);
-		response.headers().set(LAST_MODIFIED, dateFormatter.format(new Date(fileToCache.lastModified())));
-	}
-
-	/**
-	 * Sets the content type header for the HTTP Response.
-	 *
-	 * @param response HTTP response
-	 * @param file     file to extract content type
-	 */
-	public static void setContentTypeHeader(HttpResponse response, File file) {
-		String mimeType = MimeTypes.getMimeTypeForFileName(file.getName());
-		String mimeFinal = mimeType != null ? mimeType : MimeTypes.getDefaultMimeType();
-		response.headers().set(CONTENT_TYPE, mimeFinal);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
deleted file mode 100644
index 053d3f7..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
+++ /dev/null
@@ -1,80 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.NotFoundException;
-import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for request handlers whose response depends on an ExecutionGraph
- * that can be retrieved via "jobid" parameter.
- */
-public abstract class AbstractExecutionGraphRequestHandler extends AbstractJsonRequestHandler {
-
-	private final ExecutionGraphHolder executionGraphHolder;
-
-	public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executor);
-		this.executionGraphHolder = Preconditions.checkNotNull(executionGraphHolder);
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(
-			Map<String, String> pathParams,
-			Map<String, String> queryParams,
-			JobManagerGateway jobManagerGateway) {
-		String jidString = pathParams.get("jobid");
-		if (jidString == null) {
-			throw new RuntimeException("JobId parameter missing");
-		}
-
-		JobID jid;
-		try {
-			jid = JobID.fromHexString(jidString);
-		}
-		catch (Exception e) {
-			return FutureUtils.completedExceptionally(new FlinkException("Invalid JobID string '" + jidString + "'", e));
-		}
-
-		final CompletableFuture<Optional<AccessExecutionGraph>> graphFuture = executionGraphHolder.getExecutionGraph(jid, jobManagerGateway);
-
-		return graphFuture.thenComposeAsync(
-			(Optional<AccessExecutionGraph> optGraph) -> {
-				if (optGraph.isPresent()) {
-					return handleRequest(optGraph.get(), pathParams);
-				} else {
-					throw new FlinkFutureException(new NotFoundException("Could not find job with jobId " + jid + '.'));
-				}
-			}, executor);
-	}
-
-	public abstract CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java
deleted file mode 100644
index df09225..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java
+++ /dev/null
@@ -1,72 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for request handlers whose response depends on a specific job vertex (defined
- * via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
- */
-public abstract class AbstractJobVertexRequestHandler extends AbstractExecutionGraphRequestHandler {
-
-	public AbstractJobVertexRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public final CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		final JobVertexID vid = parseJobVertexId(params);
-
-		final AccessExecutionJobVertex jobVertex = graph.getJobVertex(vid);
-		if (jobVertex == null) {
-			throw new IllegalArgumentException("No vertex with ID '" + vid + "' exists.");
-		}
-
-		return handleRequest(jobVertex, params);
-	}
-
-	/**
-	 * Returns the job vertex ID parsed from the provided parameters.
-	 *
-	 * @param params Path parameters
-	 * @return Parsed job vertex ID or <code>null</code> if not available.
-	 */
-	public static JobVertexID parseJobVertexId(Map<String, String> params) {
-		String jobVertexIdParam = params.get("vertexid");
-		if (jobVertexIdParam == null) {
-			return null;
-		}
-
-		try {
-			return JobVertexID.fromHexString(jobVertexIdParam);
-		} catch (RuntimeException ignored) {
-			return null;
-		}
-	}
-
-	public abstract CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params);
-}


[03/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandlerTest.java
new file mode 100644
index 0000000..3783b84
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandlerTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.TestLogger;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SubtasksTimesHandler.
+ */
+public class SubtasksTimesHandlerTest extends TestLogger {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new SubtasksTimesHandler.SubtasksTimesJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/subtasktimes", archive.getPath());
+		compareSubtaskTimes(originalTask, originalAttempt, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		SubtasksTimesHandler handler = new SubtasksTimesHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasktimes", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+		String json = SubtasksTimesHandler.createSubtaskTimesJson(originalTask);
+
+		compareSubtaskTimes(originalTask, originalAttempt, json);
+	}
+
+	private static void compareSubtaskTimes(AccessExecutionJobVertex originalTask, AccessExecution originalAttempt, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
+		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
+		Assert.assertTrue(result.get("now").asLong() > 0L);
+
+		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
+
+		JsonNode subtask = subtasks.get(0);
+		Assert.assertEquals(0, subtask.get("subtask").asInt());
+		Assert.assertEquals(originalAttempt.getAssignedResourceLocation().getHostname(), subtask.get("host").asText());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(originalAttempt.getState()) - originalAttempt.getStateTimestamp(ExecutionState.SCHEDULED), subtask.get("duration").asLong());
+
+		JsonNode timestamps = subtask.get("timestamps");
+
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CREATED), timestamps.get(ExecutionState.CREATED.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.SCHEDULED), timestamps.get(ExecutionState.SCHEDULED.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.DEPLOYING), timestamps.get(ExecutionState.DEPLOYING.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.RUNNING), timestamps.get(ExecutionState.RUNNING.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.FINISHED), timestamps.get(ExecutionState.FINISHED.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CANCELING), timestamps.get(ExecutionState.CANCELING.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CANCELED), timestamps.get(ExecutionState.CANCELED.name()).asLong());
+		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.FAILED), timestamps.get(ExecutionState.FAILED.name()).asLong());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java
new file mode 100644
index 0000000..b65dcb6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/**
+ * Tests for the TaskManagersLogHandler.
+ */
+public class TaskManagerLogHandlerTest {
+	@Test
+	public void testGetPaths() {
+		TaskManagerLogHandler handlerLog = new TaskManagerLogHandler(
+			mock(GatewayRetriever.class),
+			Executors.directExecutor(),
+			CompletableFuture.completedFuture("/jm/address"),
+			TestingUtils.TIMEOUT(),
+			TaskManagerLogHandler.FileMode.LOG,
+			new Configuration(),
+			new VoidBlobStore());
+		String[] pathsLog = handlerLog.getPaths();
+		Assert.assertEquals(1, pathsLog.length);
+		Assert.assertEquals("/taskmanagers/:taskmanagerid/log", pathsLog[0]);
+
+		TaskManagerLogHandler handlerOut = new TaskManagerLogHandler(
+			mock(GatewayRetriever.class),
+			Executors.directExecutor(),
+			CompletableFuture.completedFuture("/jm/address"),
+			TestingUtils.TIMEOUT(),
+			TaskManagerLogHandler.FileMode.STDOUT,
+			new Configuration(),
+			new VoidBlobStore());
+		String[] pathsOut = handlerOut.getPaths();
+		Assert.assertEquals(1, pathsOut.length);
+		Assert.assertEquals("/taskmanagers/:taskmanagerid/stdout", pathsOut[0]);
+	}
+
+	@Test
+	public void testLogFetchingFailure() throws Exception {
+		// ========= setup TaskManager =================================================================================
+		InstanceID tmID = new InstanceID();
+		ResourceID tmRID = new ResourceID(tmID.toString());
+		TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class);
+		when(taskManagerGateway.getAddress()).thenReturn("/tm/address");
+
+		Instance taskManager = mock(Instance.class);
+		when(taskManager.getId()).thenReturn(tmID);
+		when(taskManager.getTaskManagerID()).thenReturn(tmRID);
+		when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway);
+		CompletableFuture<BlobKey> future = new CompletableFuture<>();
+		future.completeExceptionally(new IOException("failure"));
+		when(taskManagerGateway.requestTaskManagerLog(any(Time.class))).thenReturn(future);
+
+		// ========= setup JobManager ==================================================================================
+
+		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
+		when(jobManagerGateway.requestBlobServerPort(any(Time.class))).thenReturn(CompletableFuture.completedFuture(1337));
+		when(jobManagerGateway.getHostname()).thenReturn("localhost");
+		when(jobManagerGateway.requestTaskManagerInstance(any(InstanceID.class), any(Time.class))).thenReturn(
+			CompletableFuture.completedFuture(Optional.of(taskManager)));
+
+		GatewayRetriever<JobManagerGateway> retriever = mock(GatewayRetriever.class);
+		when(retriever.getNow())
+			.thenReturn(Optional.of(jobManagerGateway));
+
+		TaskManagerLogHandler handler = new TaskManagerLogHandler(
+			retriever,
+			Executors.directExecutor(),
+			CompletableFuture.completedFuture("/jm/address"),
+			TestingUtils.TIMEOUT(),
+			TaskManagerLogHandler.FileMode.LOG,
+			new Configuration(),
+			new VoidBlobStore());
+
+		final AtomicReference<String> exception = new AtomicReference<>();
+
+		ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
+		when(ctx.write(isA(ByteBuf.class))).thenAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				ByteBuf data = invocationOnMock.getArgumentAt(0, ByteBuf.class);
+				exception.set(new String(data.array(), ConfigConstants.DEFAULT_CHARSET));
+				return null;
+			}
+		});
+
+		Map<String, String> pathParams = new HashMap<>();
+		pathParams.put(TaskManagersHandler.TASK_MANAGER_ID_KEY, tmID.toString());
+		Routed routed = mock(Routed.class);
+		when(routed.pathParams()).thenReturn(pathParams);
+		when(routed.request()).thenReturn(new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/taskmanagers/" + tmID + "/log"));
+
+		handler.respondAsLeader(ctx, routed, jobManagerGateway);
+
+		Assert.assertEquals("Fetching TaskManager log failed.", exception.get());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandlerTest.java
new file mode 100644
index 0000000..2f8afd1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandlerTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Executors;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Tests for the TaskManagersHandler.
+ */
+public class TaskManagersHandlerTest {
+	@Test
+	public void testGetPaths() {
+		TaskManagersHandler handler = new TaskManagersHandler(Executors.directExecutor(), Time.seconds(0L), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(2, paths.length);
+		List<String> pathsList = Lists.newArrayList(paths);
+		Assert.assertTrue(pathsList.contains("/taskmanagers"));
+		Assert.assertTrue(pathsList.contains("/taskmanagers/:taskmanagerid"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java
new file mode 100644
index 0000000..e2289f0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java
@@ -0,0 +1,329 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobClient;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
+
+/**
+ * Simple back pressured task test.
+ */
+public class BackPressureStatsTrackerITCase extends TestLogger {
+
+	private static NetworkBufferPool networkBufferPool;
+	private static ActorSystem testActorSystem;
+
+	/** Shared as static variable with the test task. */
+	private static BufferPool testBufferPool;
+
+	@BeforeClass
+	public static void setup() {
+		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
+		networkBufferPool = new NetworkBufferPool(100, 8192, MemoryType.HEAP);
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(testActorSystem);
+		networkBufferPool.destroy();
+	}
+
+	/**
+	 * Tests a simple fake-back pressured task. Back pressure is assumed when
+	 * sampled stack traces are in blocking buffer requests.
+	 */
+	@Test
+	public void testBackPressuredProducer() throws Exception {
+		new JavaTestKit(testActorSystem) {{
+			final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS);
+
+			// The JobGraph
+			final JobGraph jobGraph = new JobGraph();
+			final int parallelism = 4;
+
+			final JobVertex task = new JobVertex("Task");
+			task.setInvokableClass(BackPressuredTask.class);
+			task.setParallelism(parallelism);
+
+			jobGraph.addVertex(task);
+
+			final Configuration config = new Configuration();
+
+			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
+			ActorGateway jobManger = null;
+			ActorGateway taskManager = null;
+
+			//
+			// 1) Consume all buffers at first (no buffers for the test task)
+			//
+			testBufferPool = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE);
+			final List<Buffer> buffers = new ArrayList<>();
+			while (true) {
+				Buffer buffer = testBufferPool.requestBuffer();
+				if (buffer != null) {
+					buffers.add(buffer);
+				} else {
+					break;
+				}
+			}
+
+			try {
+				jobManger = TestingUtils.createJobManager(
+					testActorSystem,
+					TestingUtils.defaultExecutor(),
+					TestingUtils.defaultExecutor(),
+					config,
+					highAvailabilityServices);
+
+				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
+
+				taskManager = TestingUtils.createTaskManager(
+					testActorSystem,
+					highAvailabilityServices,
+					config,
+					true,
+					true);
+
+				final ActorGateway jm = jobManger;
+
+				new Within(deadline) {
+					@Override
+					protected void run() {
+						try {
+							ActorGateway testActor = new AkkaActorGateway(getTestActor(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+							// Submit the job and wait until it is running
+							JobClient.submitJobDetached(
+									new AkkaJobManagerGateway(jm),
+									config,
+									jobGraph,
+									Time.milliseconds(deadline.toMillis()),
+									ClassLoader.getSystemClassLoader());
+
+							jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
+
+							expectMsgEquals(new AllVerticesRunning(jobGraph.getJobID()));
+
+							// Get the ExecutionGraph
+							jm.tell(new RequestExecutionGraph(jobGraph.getJobID()), testActor);
+
+							ExecutionGraphFound executionGraphResponse =
+									expectMsgClass(ExecutionGraphFound.class);
+
+							ExecutionGraph executionGraph = (ExecutionGraph) executionGraphResponse.executionGraph();
+							ExecutionJobVertex vertex = executionGraph.getJobVertex(task.getID());
+
+							StackTraceSampleCoordinator coordinator = new StackTraceSampleCoordinator(
+									testActorSystem.dispatcher(), 60000);
+
+							// Verify back pressure (clean up interval can be ignored)
+							BackPressureStatsTracker statsTracker = new BackPressureStatsTracker(
+								coordinator,
+								100 * 1000,
+								20,
+								Time.milliseconds(10L));
+
+							int numAttempts = 10;
+
+							int nextSampleId = 0;
+
+							// Verify that all tasks are back pressured. This
+							// can fail if the task takes longer to request
+							// the buffer.
+							for (int attempt = 0; attempt < numAttempts; attempt++) {
+								try {
+									OperatorBackPressureStats stats = triggerStatsSample(statsTracker, vertex);
+
+									Assert.assertEquals(nextSampleId + attempt, stats.getSampleId());
+									Assert.assertEquals(parallelism, stats.getNumberOfSubTasks());
+									Assert.assertEquals(1.0, stats.getMaxBackPressureRatio(), 0.0);
+
+									for (int i = 0; i < parallelism; i++) {
+										Assert.assertEquals(1.0, stats.getBackPressureRatio(i), 0.0);
+									}
+
+									nextSampleId = stats.getSampleId() + 1;
+
+									break;
+								} catch (Throwable t) {
+									if (attempt == numAttempts - 1) {
+										throw t;
+									} else {
+										Thread.sleep(500);
+									}
+								}
+							}
+
+							//
+							// 2) Release all buffers and let the tasks grab one
+							//
+							for (Buffer buf : buffers) {
+								buf.recycle();
+							}
+
+							// Wait for all buffers to be available. The tasks
+							// grab them and then immediately release them.
+							while (testBufferPool.getNumberOfAvailableMemorySegments() < 100) {
+								Thread.sleep(100);
+							}
+
+							// Verify that no task is back pressured any more.
+							for (int attempt = 0; attempt < numAttempts; attempt++) {
+								try {
+									OperatorBackPressureStats stats = triggerStatsSample(statsTracker, vertex);
+
+									Assert.assertEquals(nextSampleId + attempt, stats.getSampleId());
+									Assert.assertEquals(parallelism, stats.getNumberOfSubTasks());
+
+									// Verify that no task is back pressured
+									for (int i = 0; i < parallelism; i++) {
+										Assert.assertEquals(0.0, stats.getBackPressureRatio(i), 0.0);
+									}
+
+									break;
+								} catch (Throwable t) {
+									if (attempt == numAttempts - 1) {
+										throw t;
+									} else {
+										Thread.sleep(500);
+									}
+								}
+							}
+
+							// Shut down
+							jm.tell(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobGraph.getJobID()), testActor);
+
+							// Cancel job
+							jm.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
+
+							// Response to removal notification
+							expectMsgEquals(true);
+
+							//
+							// 3) Trigger stats for archived job
+							//
+							statsTracker.invalidateOperatorStatsCache();
+							Assert.assertFalse("Unexpected trigger", statsTracker.triggerStackTraceSample(vertex));
+
+						} catch (Exception e) {
+							e.printStackTrace();
+							Assert.fail(e.getMessage());
+						}
+					}
+				};
+			} finally {
+				TestingUtils.stopActor(jobManger);
+				TestingUtils.stopActor(taskManager);
+
+				highAvailabilityServices.closeAndCleanupAllData();
+
+				for (Buffer buf : buffers) {
+					buf.recycle();
+				}
+
+				testBufferPool.lazyDestroy();
+			}
+		}};
+	}
+
+	/**
+	 * Triggers a new stats sample.
+	 */
+	private OperatorBackPressureStats triggerStatsSample(
+			BackPressureStatsTracker statsTracker,
+			ExecutionJobVertex vertex) throws InterruptedException {
+
+		statsTracker.invalidateOperatorStatsCache();
+		Assert.assertTrue("Failed to trigger", statsTracker.triggerStackTraceSample(vertex));
+
+		// Sleep minimum duration
+		Thread.sleep(20 * 10);
+
+		Optional<OperatorBackPressureStats> stats;
+
+		// Get the stats
+		while (!(stats = statsTracker.getOperatorBackPressureStats(vertex)).isPresent()) {
+			Thread.sleep(10);
+		}
+
+		return stats.get();
+	}
+
+	/**
+	 * A back pressured producer sharing a {@link BufferPool} with the
+	 * test driver.
+	 */
+	public static class BackPressuredTask extends AbstractInvokable {
+
+		@Override
+		public void invoke() throws Exception {
+			while (true) {
+				Buffer buffer = testBufferPool.requestBufferBlocking();
+				// Got a buffer, yay!
+				buffer.recycle();
+
+				new CountDownLatch(1).await();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerTest.java
new file mode 100644
index 0000000..02f954a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Tests for the BackPressureStatsTracker.
+ */
+public class BackPressureStatsTrackerTest extends TestLogger {
+
+	/** Tests simple statistics with fake stack traces. */
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testTriggerStackTraceSample() throws Exception {
+		CompletableFuture<StackTraceSample> sampleFuture = new CompletableFuture<>();
+
+		StackTraceSampleCoordinator sampleCoordinator = Mockito.mock(StackTraceSampleCoordinator.class);
+		Mockito.when(sampleCoordinator.triggerStackTraceSample(
+				Matchers.any(ExecutionVertex[].class),
+				Matchers.anyInt(),
+				Matchers.any(Time.class),
+				Matchers.anyInt())).thenReturn(sampleFuture);
+
+		ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
+		Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
+
+		// Same Thread execution context
+		Mockito.when(graph.getFutureExecutor()).thenReturn(new Executor() {
+
+			@Override
+			public void execute(Runnable runnable) {
+				runnable.run();
+			}
+		});
+
+		ExecutionVertex[] taskVertices = new ExecutionVertex[4];
+
+		ExecutionJobVertex jobVertex = Mockito.mock(ExecutionJobVertex.class);
+		Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
+		Mockito.when(jobVertex.getJobVertexId()).thenReturn(new JobVertexID());
+		Mockito.when(jobVertex.getGraph()).thenReturn(graph);
+		Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
+
+		taskVertices[0] = mockExecutionVertex(jobVertex, 0);
+		taskVertices[1] = mockExecutionVertex(jobVertex, 1);
+		taskVertices[2] = mockExecutionVertex(jobVertex, 2);
+		taskVertices[3] = mockExecutionVertex(jobVertex, 3);
+
+		int numSamples = 100;
+		Time delayBetweenSamples = Time.milliseconds(100L);
+
+		BackPressureStatsTracker tracker = new BackPressureStatsTracker(
+				sampleCoordinator, 9999, numSamples, delayBetweenSamples);
+
+		// Trigger
+		Assert.assertTrue("Failed to trigger", tracker.triggerStackTraceSample(jobVertex));
+
+		Mockito.verify(sampleCoordinator).triggerStackTraceSample(
+				Matchers.eq(taskVertices),
+				Matchers.eq(numSamples),
+				Matchers.eq(delayBetweenSamples),
+				Matchers.eq(BackPressureStatsTracker.MAX_STACK_TRACE_DEPTH));
+
+		// Trigger again for pending request, should not fire
+		Assert.assertFalse("Unexpected trigger", tracker.triggerStackTraceSample(jobVertex));
+
+		Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
+
+		Mockito.verify(sampleCoordinator).triggerStackTraceSample(
+				Matchers.eq(taskVertices),
+				Matchers.eq(numSamples),
+				Matchers.eq(delayBetweenSamples),
+				Matchers.eq(BackPressureStatsTracker.MAX_STACK_TRACE_DEPTH));
+
+		Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
+
+		// Complete the future
+		Map<ExecutionAttemptID, List<StackTraceElement[]>> traces = new HashMap<>();
+		for (ExecutionVertex vertex : taskVertices) {
+			List<StackTraceElement[]> taskTraces = new ArrayList<>();
+
+			for (int i = 0; i < taskVertices.length; i++) {
+				// Traces until sub task index are back pressured
+				taskTraces.add(createStackTrace(i <= vertex.getParallelSubtaskIndex()));
+			}
+
+			traces.put(vertex.getCurrentExecutionAttempt().getAttemptId(), taskTraces);
+		}
+
+		int sampleId = 1231;
+		int endTime = 841;
+
+		StackTraceSample sample = new StackTraceSample(
+				sampleId,
+				0,
+				endTime,
+				traces);
+
+		// Succeed the promise
+		sampleFuture.complete(sample);
+
+		Assert.assertTrue(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
+
+		OperatorBackPressureStats stats = tracker.getOperatorBackPressureStats(jobVertex).get();
+
+		// Verify the stats
+		Assert.assertEquals(sampleId, stats.getSampleId());
+		Assert.assertEquals(endTime, stats.getEndTimestamp());
+		Assert.assertEquals(taskVertices.length, stats.getNumberOfSubTasks());
+
+		for (int i = 0; i < taskVertices.length; i++) {
+			double ratio = stats.getBackPressureRatio(i);
+			// Traces until sub task index are back pressured
+			Assert.assertEquals((i + 1) / ((double) 4), ratio, 0.0);
+		}
+	}
+
+	private StackTraceElement[] createStackTrace(boolean isBackPressure) {
+		if (isBackPressure) {
+			return new StackTraceElement[] { new StackTraceElement(
+					BackPressureStatsTracker.EXPECTED_CLASS_NAME,
+					BackPressureStatsTracker.EXPECTED_METHOD_NAME,
+					"LocalBufferPool.java",
+					133) };
+		} else {
+			return Thread.currentThread().getStackTrace();
+		}
+	}
+
+	private ExecutionVertex mockExecutionVertex(
+			ExecutionJobVertex jobVertex,
+			int subTaskIndex) {
+
+		Execution exec = Mockito.mock(Execution.class);
+		Mockito.when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID());
+
+		JobVertexID id = jobVertex.getJobVertexId();
+
+		ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class);
+		Mockito.when(vertex.getJobvertexId()).thenReturn(id);
+		Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
+		Mockito.when(vertex.getParallelSubtaskIndex()).thenReturn(subTaskIndex);
+
+		return vertex;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorITCase.java
new file mode 100644
index 0000000..8fa302a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorITCase.java
@@ -0,0 +1,203 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobClient;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
+import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
+
+/**
+ * Simple stack trace sampling test.
+ */
+public class StackTraceSampleCoordinatorITCase extends TestLogger {
+
+	private static ActorSystem testActorSystem;
+
+	@BeforeClass
+	public static void setup() {
+		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(testActorSystem);
+	}
+
+	/**
+	 * Tests that a cleared task is answered with a partial success response.
+	 */
+	@Test
+	public void testTaskClearedWhileSampling() throws Exception {
+		new JavaTestKit(testActorSystem) {{
+			final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS);
+
+			// The JobGraph
+			final JobGraph jobGraph = new JobGraph();
+			final int parallelism = 1;
+
+			final JobVertex task = new JobVertex("Task");
+			task.setInvokableClass(BlockingNoOpInvokable.class);
+			task.setParallelism(parallelism);
+
+			jobGraph.addVertex(task);
+
+			final Configuration config = new Configuration();
+
+			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
+			ActorGateway jobManger = null;
+			ActorGateway taskManager = null;
+
+			try {
+				jobManger = TestingUtils.createJobManager(
+					testActorSystem,
+					TestingUtils.defaultExecutor(),
+					TestingUtils.defaultExecutor(),
+					config,
+					highAvailabilityServices);
+
+				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
+
+				taskManager = TestingUtils.createTaskManager(
+					testActorSystem,
+					highAvailabilityServices,
+					config,
+					true,
+					true);
+
+				final ActorGateway jm = jobManger;
+
+				new Within(deadline) {
+					@Override
+					protected void run() {
+						try {
+							ActorGateway testActor = new AkkaActorGateway(getTestActor(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+							int maxAttempts = 10;
+							int sleepTime = 100;
+
+							for (int i = 0; i < maxAttempts; i++, sleepTime *= 2) {
+								// Submit the job and wait until it is running
+								JobClient.submitJobDetached(
+										new AkkaJobManagerGateway(jm),
+										config,
+										jobGraph,
+										Time.milliseconds(deadline.toMillis()),
+										ClassLoader.getSystemClassLoader());
+
+								jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
+
+								expectMsgEquals(new AllVerticesRunning(jobGraph.getJobID()));
+
+								// Get the ExecutionGraph
+								jm.tell(new RequestExecutionGraph(jobGraph.getJobID()), testActor);
+								ExecutionGraphFound executionGraphResponse =
+										expectMsgClass(ExecutionGraphFound.class);
+								ExecutionGraph executionGraph = (ExecutionGraph) executionGraphResponse.executionGraph();
+								ExecutionJobVertex vertex = executionGraph.getJobVertex(task.getID());
+
+								StackTraceSampleCoordinator coordinator = new StackTraceSampleCoordinator(
+										testActorSystem.dispatcher(), 60000);
+
+								CompletableFuture<StackTraceSample> sampleFuture = coordinator.triggerStackTraceSample(
+									vertex.getTaskVertices(),
+									// Do this often so we have a good
+									// chance of removing the job during
+									// sampling.
+									21474700 * 100,
+									Time.milliseconds(10L),
+									0);
+
+								// Wait before cancelling so that some samples
+								// are actually taken.
+								Thread.sleep(sleepTime);
+
+								// Cancel job
+								Future<?> removeFuture = jm.ask(
+										new TestingJobManagerMessages.NotifyWhenJobRemoved(jobGraph.getJobID()),
+										remaining());
+
+								jm.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
+
+								try {
+									// Throws Exception on failure
+									sampleFuture.get(remaining().toMillis(), TimeUnit.MILLISECONDS);
+
+									// OK, we are done. Got the expected
+									// partial result.
+									break;
+								} catch (Throwable t) {
+									// We were too fast in cancelling the job.
+									// Fall through and retry.
+								} finally {
+									Await.ready(removeFuture, remaining());
+								}
+							}
+						} catch (Exception e) {
+							e.printStackTrace();
+							Assert.fail(e.getMessage());
+						}
+					}
+				};
+			} finally {
+				TestingUtils.stopActor(jobManger);
+				TestingUtils.stopActor(taskManager);
+
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
+		}};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java
new file mode 100644
index 0000000..786b0ae
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java
@@ -0,0 +1,432 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample;
+import org.apache.flink.runtime.messages.StackTraceSampleResponse;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test for the {@link StackTraceSampleCoordinator}.
+ */
+public class StackTraceSampleCoordinatorTest extends TestLogger {
+
+	private static ActorSystem system;
+
+	private StackTraceSampleCoordinator coord;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		system = AkkaUtils.createLocalActorSystem(new Configuration());
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (system != null) {
+			system.shutdown();
+		}
+	}
+
+	@Before
+	public void init() throws Exception {
+		this.coord = new StackTraceSampleCoordinator(system.dispatcher(), 60000);
+	}
+
+	/** Tests simple trigger and collect of stack trace samples. */
+	@Test
+	public void testTriggerStackTraceSample() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true)
+		};
+
+		int numSamples = 1;
+		Time delayBetweenSamples = Time.milliseconds(100L);
+		int maxStackTraceDepth = 0;
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+				vertices, numSamples, delayBetweenSamples, maxStackTraceDepth);
+
+		// Verify messages have been sent
+		for (ExecutionVertex vertex : vertices) {
+			ExecutionAttemptID expectedExecutionId = vertex
+					.getCurrentExecutionAttempt().getAttemptId();
+
+			TriggerStackTraceSample expectedMsg = new TriggerStackTraceSample(
+					0,
+					expectedExecutionId,
+					numSamples,
+					delayBetweenSamples,
+					maxStackTraceDepth);
+
+			Mockito.verify(vertex.getCurrentExecutionAttempt())
+				.requestStackTraceSample(Matchers.eq(0), Matchers.eq(numSamples), Matchers.eq(delayBetweenSamples), Matchers.eq(maxStackTraceDepth), Matchers.any(Time.class));
+		}
+
+		Assert.assertFalse(sampleFuture.isDone());
+
+		StackTraceElement[] stackTraceSample = Thread.currentThread().getStackTrace();
+		List<StackTraceElement[]> traces = new ArrayList<>();
+		traces.add(stackTraceSample);
+		traces.add(stackTraceSample);
+		traces.add(stackTraceSample);
+
+		// Collect stack traces
+		for (int i = 0; i < vertices.length; i++) {
+			ExecutionAttemptID executionId = vertices[i].getCurrentExecutionAttempt().getAttemptId();
+			coord.collectStackTraces(0, executionId, traces);
+
+			if (i == vertices.length - 1) {
+				Assert.assertTrue(sampleFuture.isDone());
+			} else {
+				Assert.assertFalse(sampleFuture.isDone());
+			}
+		}
+
+		// Verify completed stack trace sample
+		StackTraceSample sample = sampleFuture.get();
+
+		Assert.assertEquals(0, sample.getSampleId());
+		Assert.assertTrue(sample.getEndTime() >= sample.getStartTime());
+
+		Map<ExecutionAttemptID, List<StackTraceElement[]>> tracesByTask = sample.getStackTraces();
+
+		for (ExecutionVertex vertex : vertices) {
+			ExecutionAttemptID executionId = vertex.getCurrentExecutionAttempt().getAttemptId();
+			List<StackTraceElement[]> sampleTraces = tracesByTask.get(executionId);
+
+			Assert.assertNotNull("Task not found", sampleTraces);
+			Assert.assertTrue(traces.equals(sampleTraces));
+		}
+
+		// Verify no more pending sample
+		Assert.assertEquals(0, coord.getNumberOfPendingSamples());
+
+		// Verify no error on late collect
+		coord.collectStackTraces(0, vertices[0].getCurrentExecutionAttempt().getAttemptId(), traces);
+	}
+
+	/** Tests triggering for non-running tasks fails the future. */
+	@Test
+	public void testTriggerStackTraceSampleNotRunningTasks() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.DEPLOYING, true)
+		};
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+			vertices,
+			1,
+			Time.milliseconds(100L),
+			0);
+
+		Assert.assertTrue(sampleFuture.isDone());
+
+		try {
+			sampleFuture.get();
+			Assert.fail("Expected exception.");
+		} catch (ExecutionException e) {
+			Assert.assertTrue(e.getCause() instanceof IllegalStateException);
+		}
+	}
+
+	/** Tests triggering for reset tasks fails the future. */
+	@Test(timeout = 1000L)
+	public void testTriggerStackTraceSampleResetRunningTasks() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+				// Fails to send the message to the execution (happens when execution is reset)
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, false)
+		};
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+			vertices,
+			1,
+			Time.milliseconds(100L),
+			0);
+
+		try {
+			sampleFuture.get();
+			Assert.fail("Expected exception.");
+		} catch (ExecutionException e) {
+			Assert.assertTrue(e.getCause() instanceof RuntimeException);
+		}
+	}
+
+	/** Tests that samples time out if they don't finish in time. */
+	@Test(timeout = 1000L)
+	public void testTriggerStackTraceSampleTimeout() throws Exception {
+		int timeout = 100;
+
+		coord = new StackTraceSampleCoordinator(system.dispatcher(), timeout);
+
+		final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+
+		try {
+
+			ExecutionVertex[] vertices = new ExecutionVertex[]{
+				mockExecutionVertexWithTimeout(
+					new ExecutionAttemptID(),
+					ExecutionState.RUNNING,
+					scheduledExecutorService,
+					timeout)
+			};
+
+			CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0);
+
+			// Wait for the timeout
+			Thread.sleep(timeout * 2);
+
+			boolean success = false;
+			for (int i = 0; i < 10; i++) {
+				if (sampleFuture.isDone()) {
+					success = true;
+					break;
+				}
+
+				Thread.sleep(timeout);
+			}
+
+			Assert.assertTrue("Sample did not time out", success);
+
+			try {
+				sampleFuture.get();
+				Assert.fail("Expected exception.");
+			} catch (ExecutionException e) {
+				Assert.assertTrue(e.getCause().getCause().getMessage().contains("Timeout"));
+			}
+
+			// Collect after the timeout (should be ignored)
+			ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
+			coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
+		} finally {
+			scheduledExecutorService.shutdownNow();
+		}
+	}
+
+	/** Tests that collecting an unknown sample is ignored. */
+	@Test
+	public void testCollectStackTraceForUnknownSample() throws Exception {
+		coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList<StackTraceElement[]>());
+	}
+
+	/** Tests cancelling of a pending sample. */
+	@Test
+	public void testCancelStackTraceSample() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+		};
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0);
+
+		Assert.assertFalse(sampleFuture.isDone());
+
+		// Cancel
+		coord.cancelStackTraceSample(0, null);
+
+		// Verify completed
+		Assert.assertTrue(sampleFuture.isDone());
+
+		// Verify no more pending samples
+		Assert.assertEquals(0, coord.getNumberOfPendingSamples());
+	}
+
+	/** Tests that collecting for a cancelled sample throws no Exception. */
+	@Test
+	public void testCollectStackTraceForCanceledSample() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+		};
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0);
+
+		Assert.assertFalse(sampleFuture.isDone());
+
+		coord.cancelStackTraceSample(0, null);
+
+		Assert.assertTrue(sampleFuture.isDone());
+
+		// Verify no error on late collect
+		ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
+		coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
+	}
+
+	/** Tests that collecting for a cancelled sample throws no Exception. */
+	@Test
+	public void testCollectForDiscardedPendingSample() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+		};
+
+		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0);
+
+		Assert.assertFalse(sampleFuture.isDone());
+
+		coord.cancelStackTraceSample(0, null);
+
+		Assert.assertTrue(sampleFuture.isDone());
+
+		// Verify no error on late collect
+		ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
+		coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
+	}
+
+
+	/** Tests that collecting for a unknown task fails. */
+	@Test(expected = IllegalArgumentException.class)
+	public void testCollectStackTraceForUnknownTask() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+		};
+
+		coord.triggerStackTraceSample(vertices, 1, Time.milliseconds(100L), 0);
+
+		coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList<StackTraceElement[]>());
+	}
+
+	/** Tests that shut down fails all pending samples and future sample triggers. */
+	@Test
+	public void testShutDown() throws Exception {
+		ExecutionVertex[] vertices = new ExecutionVertex[] {
+				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
+		};
+
+		List<CompletableFuture<StackTraceSample>> sampleFutures = new ArrayList<>();
+
+		// Trigger
+		sampleFutures.add(coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0));
+
+		sampleFutures.add(coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0));
+
+		for (CompletableFuture<StackTraceSample> future : sampleFutures) {
+			Assert.assertFalse(future.isDone());
+		}
+
+		// Shut down
+		coord.shutDown();
+
+		// Verify all completed
+		for (CompletableFuture<StackTraceSample> future : sampleFutures) {
+			Assert.assertTrue(future.isDone());
+		}
+
+		// Verify new trigger returns failed future
+		CompletableFuture<StackTraceSample> future = coord.triggerStackTraceSample(
+				vertices, 1, Time.milliseconds(100L), 0);
+
+		Assert.assertTrue(future.isDone());
+
+		try {
+			future.get();
+			Assert.fail("Expected exception.");
+		} catch (ExecutionException e) {
+			// we expected an exception here :-)
+		}
+
+	}
+
+	// ------------------------------------------------------------------------
+
+	private ExecutionVertex mockExecutionVertex(
+			ExecutionAttemptID executionId,
+			ExecutionState state,
+			boolean sendSuccess) {
+
+		Execution exec = Mockito.mock(Execution.class);
+		CompletableFuture<StackTraceSampleResponse> failedFuture = new CompletableFuture<>();
+		failedFuture.completeExceptionally(new Exception("Send failed."));
+
+		Mockito.when(exec.getAttemptId()).thenReturn(executionId);
+		Mockito.when(exec.getState()).thenReturn(state);
+		Mockito.when(exec.requestStackTraceSample(Matchers.anyInt(), Matchers.anyInt(), Matchers.any(Time.class), Matchers.anyInt(), Matchers.any(Time.class)))
+			.thenReturn(
+				sendSuccess ?
+					CompletableFuture.completedFuture(Mockito.mock(StackTraceSampleResponse.class)) :
+					failedFuture);
+
+		ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class);
+		Mockito.when(vertex.getJobvertexId()).thenReturn(new JobVertexID());
+		Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
+
+		return vertex;
+	}
+
+	private ExecutionVertex mockExecutionVertexWithTimeout(
+		ExecutionAttemptID executionId,
+		ExecutionState state,
+		ScheduledExecutorService scheduledExecutorService,
+		int timeout) {
+
+		final CompletableFuture<StackTraceSampleResponse> future = new CompletableFuture<>();
+
+		Execution exec = Mockito.mock(Execution.class);
+		Mockito.when(exec.getAttemptId()).thenReturn(executionId);
+		Mockito.when(exec.getState()).thenReturn(state);
+		Mockito.when(exec.requestStackTraceSample(Matchers.anyInt(), Matchers.anyInt(), Matchers.any(Time.class), Matchers.anyInt(), Matchers.any(Time.class)))
+			.thenReturn(future);
+
+		scheduledExecutorService.schedule(new Runnable() {
+			@Override
+			public void run() {
+				future.completeExceptionally(new TimeoutException("Timeout"));
+			}
+		}, timeout, TimeUnit.MILLISECONDS);
+
+		ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class);
+		Mockito.when(vertex.getJobvertexId()).thenReturn(new JobVertexID());
+		Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
+
+		return vertex;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
new file mode 100644
index 0000000..db91f58
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandlerTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
+import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the CheckpointConfigHandler.
+ */
+public class CheckpointConfigHandlerTest {
+
+	@Test
+	public void testArchiver() throws IOException {
+		JsonArchivist archivist = new CheckpointConfigHandler.CheckpointConfigJsonArchivist();
+		GraphAndSettings graphAndSettings = createGraphAndSettings(true, true);
+
+		AccessExecutionGraph graph = graphAndSettings.graph;
+		when(graph.getJobID()).thenReturn(new JobID());
+		JobCheckpointingSettings settings = graphAndSettings.snapshottingSettings;
+		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
+		Assert.assertEquals(1, archives.size());
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + graph.getJobID() + "/checkpoints/config", archive.getPath());
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(archive.getJson());
+
+		Assert.assertEquals("exactly_once", rootNode.get("mode").asText());
+		Assert.assertEquals(settings.getCheckpointInterval(), rootNode.get("interval").asLong());
+		Assert.assertEquals(settings.getCheckpointTimeout(), rootNode.get("timeout").asLong());
+		Assert.assertEquals(settings.getMinPauseBetweenCheckpoints(), rootNode.get("min_pause").asLong());
+		Assert.assertEquals(settings.getMaxConcurrentCheckpoints(), rootNode.get("max_concurrent").asInt());
+
+		JsonNode externalizedNode = rootNode.get("externalization");
+		Assert.assertNotNull(externalizedNode);
+		Assert.assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
+		Assert.assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
+
+	}
+
+	@Test
+	public void testGetPaths() {
+		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/checkpoints/config", paths[0]);
+	}
+
+	/**
+	 * Tests a simple config.
+	 */
+	@Test
+	public void testSimpleConfig() throws Exception {
+		GraphAndSettings graphAndSettings = createGraphAndSettings(false, true);
+
+		AccessExecutionGraph graph = graphAndSettings.graph;
+		JobCheckpointingSettings settings = graphAndSettings.snapshottingSettings;
+
+		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(json);
+
+		assertEquals("exactly_once", rootNode.get("mode").asText());
+		assertEquals(settings.getCheckpointInterval(), rootNode.get("interval").asLong());
+		assertEquals(settings.getCheckpointTimeout(), rootNode.get("timeout").asLong());
+		assertEquals(settings.getMinPauseBetweenCheckpoints(), rootNode.get("min_pause").asLong());
+		assertEquals(settings.getMaxConcurrentCheckpoints(), rootNode.get("max_concurrent").asInt());
+
+		JsonNode externalizedNode = rootNode.get("externalization");
+		assertNotNull(externalizedNode);
+		assertEquals(false, externalizedNode.get("enabled").asBoolean());
+	}
+
+	/**
+	 * Tests the that the isExactlyOnce flag is respected.
+	 */
+	@Test
+	public void testAtLeastOnce() throws Exception {
+		GraphAndSettings graphAndSettings = createGraphAndSettings(false, false);
+
+		AccessExecutionGraph graph = graphAndSettings.graph;
+
+		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(json);
+
+		assertEquals("at_least_once", rootNode.get("mode").asText());
+	}
+
+	/**
+	 * Tests that the externalized checkpoint settings are forwarded.
+	 */
+	@Test
+	public void testEnabledExternalizedCheckpointSettings() throws Exception {
+		GraphAndSettings graphAndSettings = createGraphAndSettings(true, false);
+
+		AccessExecutionGraph graph = graphAndSettings.graph;
+		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
+
+		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode externalizedNode = mapper.readTree(json).get("externalization");
+		assertNotNull(externalizedNode);
+		assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
+		assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
+	}
+
+	private static GraphAndSettings createGraphAndSettings(boolean externalized, boolean exactlyOnce) {
+		long interval = 18231823L;
+		long timeout = 996979L;
+		long minPause = 119191919L;
+		int maxConcurrent = 12929329;
+		ExternalizedCheckpointSettings externalizedSetting = externalized
+			? ExternalizedCheckpointSettings.externalizeCheckpoints(true)
+			: ExternalizedCheckpointSettings.none();
+
+		JobCheckpointingSettings settings = new JobCheckpointingSettings(
+			Collections.<JobVertexID>emptyList(),
+			Collections.<JobVertexID>emptyList(),
+			Collections.<JobVertexID>emptyList(),
+			interval,
+			timeout,
+			minPause,
+			maxConcurrent,
+			externalizedSetting,
+			null,
+			exactlyOnce);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getJobCheckpointingSettings()).thenReturn(settings);
+
+		return new GraphAndSettings(graph, settings, externalizedSetting);
+	}
+
+	private static class GraphAndSettings {
+		public final AccessExecutionGraph graph;
+		public final JobCheckpointingSettings snapshottingSettings;
+		public final ExternalizedCheckpointSettings externalizedSettings;
+
+		public GraphAndSettings(
+				AccessExecutionGraph graph,
+				JobCheckpointingSettings snapshottingSettings,
+				ExternalizedCheckpointSettings externalizedSettings) {
+			this.graph = graph;
+			this.snapshottingSettings = snapshottingSettings;
+			this.externalizedSettings = externalizedSettings;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCacheTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCacheTest.java
new file mode 100644
index 0000000..04b1c55
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCacheTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the CheckpoitnStatsCache.
+ */
+public class CheckpointStatsCacheTest {
+
+	@Test
+	public void testZeroSizeCache() throws Exception {
+		AbstractCheckpointStats checkpoint = createCheckpoint(0, CheckpointStatsStatus.COMPLETED);
+
+		CheckpointStatsCache cache = new CheckpointStatsCache(0);
+		cache.tryAdd(checkpoint);
+		assertNull(cache.tryGet(0L));
+	}
+
+	@Test
+	public void testCacheAddAndGet() throws Exception {
+		AbstractCheckpointStats chk0 = createCheckpoint(0, CheckpointStatsStatus.COMPLETED);
+		AbstractCheckpointStats chk1 = createCheckpoint(1, CheckpointStatsStatus.COMPLETED);
+		AbstractCheckpointStats chk2 = createCheckpoint(2, CheckpointStatsStatus.IN_PROGRESS);
+
+		CheckpointStatsCache cache = new CheckpointStatsCache(1);
+		cache.tryAdd(chk0);
+		assertEquals(chk0, cache.tryGet(0));
+
+		cache.tryAdd(chk1);
+		assertNull(cache.tryGet(0));
+		assertEquals(chk1, cache.tryGet(1));
+
+		cache.tryAdd(chk2);
+		assertNull(cache.tryGet(2));
+		assertNull(cache.tryGet(0));
+		assertEquals(chk1, cache.tryGet(1));
+	}
+
+	private AbstractCheckpointStats createCheckpoint(long id, CheckpointStatsStatus status) {
+		AbstractCheckpointStats checkpoint = mock(AbstractCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(id);
+		when(checkpoint.getStatus()).thenReturn(status);
+		return checkpoint;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
new file mode 100644
index 0000000..e614608
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandlerTest.java
@@ -0,0 +1,358 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
+import org.apache.flink.runtime.checkpoint.TaskStateStats;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the CheckpointStatsDetailsHandler.
+ */
+public class CheckpointStatsDetailsHandlerTest {
+
+	@Test
+	public void testArchiver() throws IOException {
+		JsonArchivist archivist = new CheckpointStatsDetailsHandler.CheckpointStatsDetailsJsonArchivist();
+
+		CompletedCheckpointStats completedCheckpoint = createCompletedCheckpoint();
+		FailedCheckpointStats failedCheckpoint = createFailedCheckpoint();
+		List<AbstractCheckpointStats> checkpoints = new ArrayList<>();
+		checkpoints.add(failedCheckpoint);
+		checkpoints.add(completedCheckpoint);
+
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpoints()).thenReturn(checkpoints);
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+		when(graph.getJobID()).thenReturn(new JobID());
+
+		ObjectMapper mapper = new ObjectMapper();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
+		Assert.assertEquals(2, archives.size());
+
+		Iterator<ArchivedJson> iterator = archives.iterator();
+		ArchivedJson archive1 = iterator.next();
+		Assert.assertEquals(
+			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + failedCheckpoint.getCheckpointId(),
+			archive1.getPath());
+		compareFailedCheckpoint(failedCheckpoint, mapper.readTree(archive1.getJson()));
+
+		ArchivedJson archive2 = iterator.next();
+		Assert.assertEquals(
+			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + completedCheckpoint.getCheckpointId(),
+			archive2.getPath());
+		compareCompletedCheckpoint(completedCheckpoint, mapper.readTree(archive2.getJson()));
+	}
+
+	@Test
+	public void testGetPaths() {
+		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid", paths[0]);
+	}
+
+	/**
+	 * Tests request with illegal checkpoint ID param.
+	 */
+	@Test
+	public void testIllegalCheckpointId() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "illegal checkpoint");
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Tests request with missing checkpoint ID param.
+	 */
+	@Test
+	public void testNoCheckpointIdParam() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Test lookup of not existing checkpoint in history.
+	 */
+	@Test
+	public void testCheckpointNotFound() throws Exception {
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpointById(anyLong())).thenReturn(null); // not found
+
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "123");
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+		verify(history, times(1)).getCheckpointById(anyLong());
+	}
+
+	/**
+	 * Tests a checkpoint details request for an in progress checkpoint.
+	 */
+	@Test
+	public void testCheckpointDetailsRequestInProgressCheckpoint() throws Exception {
+		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
+		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(checkpoint.getTriggerTimestamp()).thenReturn(1919191900L);
+		when(checkpoint.getLatestAckTimestamp()).thenReturn(1977791901L);
+		when(checkpoint.getStateSize()).thenReturn(111939272822L);
+		when(checkpoint.getEndToEndDuration()).thenReturn(121191L);
+		when(checkpoint.getAlignmentBuffered()).thenReturn(1L);
+		when(checkpoint.getNumberOfSubtasks()).thenReturn(501);
+		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(101);
+
+		List<TaskStateStats> taskStats = new ArrayList<>();
+		TaskStateStats task1 = createTaskStateStats();
+		TaskStateStats task2 = createTaskStateStats();
+		taskStats.add(task1);
+		taskStats.add(task2);
+
+		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
+
+		JsonNode rootNode = triggerRequest(checkpoint);
+
+		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
+		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
+		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
+		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
+		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
+		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
+		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
+		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
+		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
+		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
+
+		verifyTaskNodes(taskStats, rootNode);
+	}
+
+	/**
+	 * Tests a checkpoint details request for a completed checkpoint.
+	 */
+	@Test
+	public void testCheckpointDetailsRequestCompletedCheckpoint() throws Exception {
+		CompletedCheckpointStats checkpoint = createCompletedCheckpoint();
+
+		JsonNode rootNode = triggerRequest(checkpoint);
+
+		compareCompletedCheckpoint(checkpoint, rootNode);
+
+		verifyTaskNodes(checkpoint.getAllTaskStateStats(), rootNode);
+	}
+
+	/**
+	 * Tests a checkpoint details request for a failed checkpoint.
+	 */
+	@Test
+	public void testCheckpointDetailsRequestFailedCheckpoint() throws Exception {
+		FailedCheckpointStats checkpoint = createFailedCheckpoint();
+
+		JsonNode rootNode = triggerRequest(checkpoint);
+
+		compareFailedCheckpoint(checkpoint, rootNode);
+
+		verifyTaskNodes(checkpoint.getAllTaskStateStats(), rootNode);
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static CompletedCheckpointStats createCompletedCheckpoint() {
+		CompletedCheckpointStats checkpoint = mock(CompletedCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1818213L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
+		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
+		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
+		when(checkpoint.getStateSize()).thenReturn(925281L);
+		when(checkpoint.getEndToEndDuration()).thenReturn(181819L);
+		when(checkpoint.getAlignmentBuffered()).thenReturn(1010198L);
+		when(checkpoint.getNumberOfSubtasks()).thenReturn(181271);
+		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(29821);
+		when(checkpoint.isDiscarded()).thenReturn(true);
+		when(checkpoint.getExternalPath()).thenReturn("checkpoint-external-path");
+
+		List<TaskStateStats> taskStats = new ArrayList<>();
+		TaskStateStats task1 = createTaskStateStats();
+		TaskStateStats task2 = createTaskStateStats();
+		taskStats.add(task1);
+		taskStats.add(task2);
+
+		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
+
+		return checkpoint;
+	}
+
+	private static void compareCompletedCheckpoint(CompletedCheckpointStats checkpoint, JsonNode rootNode) {
+		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
+		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
+		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
+		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
+		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
+		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
+		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
+		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
+		assertEquals(checkpoint.isDiscarded(), rootNode.get("discarded").asBoolean());
+		assertEquals(checkpoint.getExternalPath(), rootNode.get("external_path").asText());
+		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
+		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
+	}
+
+	private static FailedCheckpointStats createFailedCheckpoint() {
+		FailedCheckpointStats checkpoint = mock(FailedCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1818214L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
+		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
+		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
+		when(checkpoint.getStateSize()).thenReturn(925281L);
+		when(checkpoint.getEndToEndDuration()).thenReturn(181819L);
+		when(checkpoint.getAlignmentBuffered()).thenReturn(1010198L);
+		when(checkpoint.getNumberOfSubtasks()).thenReturn(181271);
+		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(29821);
+		when(checkpoint.getFailureTimestamp()).thenReturn(123012890312093L);
+		when(checkpoint.getFailureMessage()).thenReturn("failure-message");
+
+		List<TaskStateStats> taskStats = new ArrayList<>();
+		TaskStateStats task1 = createTaskStateStats();
+		TaskStateStats task2 = createTaskStateStats();
+		taskStats.add(task1);
+		taskStats.add(task2);
+
+		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
+
+		return checkpoint;
+	}
+
+	private static void compareFailedCheckpoint(FailedCheckpointStats checkpoint, JsonNode rootNode) {
+		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
+		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
+		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
+		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
+		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
+		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
+		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
+		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
+		assertEquals(checkpoint.getFailureTimestamp(), rootNode.get("failure_timestamp").asLong());
+		assertEquals(checkpoint.getFailureMessage(), rootNode.get("failure_message").asText());
+		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
+		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
+	}
+
+	private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throws Exception {
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpointById(anyLong())).thenReturn(checkpoint);
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "123");
+		String json = handler.handleRequest(graph, params).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		return mapper.readTree(json);
+	}
+
+	private static void verifyTaskNodes(Collection<TaskStateStats> tasks, JsonNode parentNode) {
+		for (TaskStateStats task : tasks) {
+			long duration = ThreadLocalRandom.current().nextInt(128);
+
+			JsonNode taskNode = parentNode.get("tasks").get(task.getJobVertexId().toString());
+			assertEquals(task.getLatestAckTimestamp(), taskNode.get("latest_ack_timestamp").asLong());
+			assertEquals(task.getStateSize(), taskNode.get("state_size").asLong());
+			assertEquals(task.getEndToEndDuration(task.getLatestAckTimestamp() - duration), taskNode.get("end_to_end_duration").asLong());
+			assertEquals(task.getAlignmentBuffered(), taskNode.get("alignment_buffered").asLong());
+			assertEquals(task.getNumberOfSubtasks(), taskNode.get("num_subtasks").asInt());
+			assertEquals(task.getNumberOfAcknowledgedSubtasks(), taskNode.get("num_acknowledged_subtasks").asInt());
+		}
+	}
+
+	private static TaskStateStats createTaskStateStats() {
+		ThreadLocalRandom rand = ThreadLocalRandom.current();
+
+		TaskStateStats task = mock(TaskStateStats.class);
+		when(task.getJobVertexId()).thenReturn(new JobVertexID());
+		when(task.getLatestAckTimestamp()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getStateSize()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getAlignmentBuffered()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getNumberOfSubtasks()).thenReturn(rand.nextInt(1024) + 1);
+		when(task.getNumberOfAcknowledgedSubtasks()).thenReturn(rand.nextInt(1024) + 1);
+		return task;
+	}
+}


[14/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
deleted file mode 100644
index cb6d8c0..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
+++ /dev/null
@@ -1,87 +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.webmonitor.handlers;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Returns the Job Manager's configuration.
- */
-public class JobManagerConfigHandler extends AbstractJsonRequestHandler {
-
-	private static final String JOBMANAGER_CONFIG_REST_PATH = "/jobmanager/config";
-
-	private final Configuration config;
-
-	public JobManagerConfigHandler(Executor executor, Configuration config) {
-		super(executor);
-		this.config = config;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOBMANAGER_CONFIG_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					StringWriter writer = new StringWriter();
-					JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-					gen.writeStartArray();
-					for (String key : config.keySet()) {
-						gen.writeStartObject();
-						gen.writeStringField("key", key);
-
-						// Mask key values which contain sensitive information
-						if (key.toLowerCase().contains("password")) {
-							String value = config.getString(key, null);
-							if (value != null) {
-								value = "******";
-							}
-							gen.writeStringField("value", value);
-						} else {
-							gen.writeStringField("value", config.getString(key, null));
-						}
-						gen.writeEndObject();
-					}
-					gen.writeEndArray();
-
-					gen.close();
-					return writer.toString();
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not write configuration.", e);
-				}
-			},
-			executor);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java
deleted file mode 100644
index b3a9dd5..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java
+++ /dev/null
@@ -1,67 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the JSON program plan of a job graph.
- */
-public class JobPlanHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String JOB_PLAN_REST_PATH = "/jobs/:jobid/plan";
-
-	public JobPlanHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_PLAN_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.completedFuture(graph.getJsonPlan());
-	}
-
-	/**
-	 * Archivist for the JobPlanHandler.
-	 */
-	public static class JobPlanJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String path = JOB_PLAN_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			String json = graph.getJsonPlan();
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java
deleted file mode 100644
index f63403f..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java
+++ /dev/null
@@ -1,72 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.StringUtils;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler for the STOP request.
- */
-public class JobStoppingHandler extends AbstractJsonRequestHandler {
-
-	private static final String JOB_STOPPING_REST_PATH = "/jobs/:jobid/stop";
-	private static final String JOB_STOPPING_YARN_REST_PATH = "/jobs/:jobid/yarn-stop";
-
-	private final Time timeout;
-
-	public JobStoppingHandler(Executor executor, Time timeout) {
-		super(executor);
-		this.timeout = Preconditions.checkNotNull(timeout);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_STOPPING_REST_PATH, JOB_STOPPING_YARN_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid")));
-					if (jobManagerGateway != null) {
-						jobManagerGateway.stopJob(jobId, timeout);
-						return "{}";
-					}
-					else {
-						throw new Exception("No connection to the leading JobManager.");
-					}
-				}
-				catch (Exception e) {
-					throw new FlinkFutureException("Failed to stop the job with id: "  + pathParams.get("jobid") + '.', e);
-				}
-			},
-			executor);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java
deleted file mode 100644
index 9c613ff..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java
+++ /dev/null
@@ -1,113 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the accummulators for a given vertex.
- */
-public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String JOB_VERTEX_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/accumulators";
-
-	public JobVertexAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_VERTEX_ACCUMULATORS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createVertexAccumulatorsJson(jobVertex);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create job vertex accumulators json.", e);
-				}
-			},
-			executor);
-
-	}
-
-	/**
-	 * Archivist for JobVertexAccumulatorsHandler.
-	 */
-	public static class JobVertexAccumulatorsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				String json = createVertexAccumulatorsJson(task);
-				String path = JOB_VERTEX_ACCUMULATORS_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":vertexid", task.getJobVertexId().toString());
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createVertexAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		StringifiedAccumulatorResult[] accs = jobVertex.getAggregatedUserAccumulatorsStringified();
-
-		gen.writeStartObject();
-		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
-
-		gen.writeArrayFieldStart("user-accumulators");
-		for (StringifiedAccumulatorResult acc : accs) {
-			gen.writeStartObject();
-			gen.writeStringField("name", acc.getName());
-			gen.writeStringField("type", acc.getType());
-			gen.writeStringField("value", acc.getValue());
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java
deleted file mode 100644
index 963153f..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java
+++ /dev/null
@@ -1,147 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.BackPressureStatsTracker;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.OperatorBackPressureStats;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import scala.Option;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Request handler that returns back pressure stats for a single job vertex and
- * all its sub tasks.
- */
-public class JobVertexBackPressureHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String JOB_VERTEX_BACKPRESSURE_REST_PATH = "/jobs/:jobid/vertices/:vertexid/backpressure";
-
-	/** Back pressure stats tracker. */
-	private final BackPressureStatsTracker backPressureStatsTracker;
-
-	/** Time after which stats are considered outdated. */
-	private final int refreshInterval;
-
-	public JobVertexBackPressureHandler(
-			ExecutionGraphHolder executionGraphHolder,
-			Executor executor,
-			BackPressureStatsTracker backPressureStatsTracker,
-			int refreshInterval) {
-
-		super(executionGraphHolder, executor);
-		this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker, "Stats tracker");
-		checkArgument(refreshInterval >= 0, "Negative timeout");
-		this.refreshInterval = refreshInterval;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_VERTEX_BACKPRESSURE_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(
-			AccessExecutionJobVertex accessJobVertex,
-			Map<String, String> params) {
-		if (accessJobVertex instanceof ArchivedExecutionJobVertex) {
-			return CompletableFuture.completedFuture("");
-		}
-		ExecutionJobVertex jobVertex = (ExecutionJobVertex) accessJobVertex;
-		try (StringWriter writer = new StringWriter();
-				JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
-
-			gen.writeStartObject();
-
-			Option<OperatorBackPressureStats> statsOption = backPressureStatsTracker
-					.getOperatorBackPressureStats(jobVertex);
-
-			if (statsOption.isDefined()) {
-				OperatorBackPressureStats stats = statsOption.get();
-
-				// Check whether we need to refresh
-				if (refreshInterval <= System.currentTimeMillis() - stats.getEndTimestamp()) {
-					backPressureStatsTracker.triggerStackTraceSample(jobVertex);
-					gen.writeStringField("status", "deprecated");
-				} else {
-					gen.writeStringField("status", "ok");
-				}
-
-				gen.writeStringField("backpressure-level", getBackPressureLevel(stats.getMaxBackPressureRatio()));
-				gen.writeNumberField("end-timestamp", stats.getEndTimestamp());
-
-				// Sub tasks
-				gen.writeArrayFieldStart("subtasks");
-				int numSubTasks = stats.getNumberOfSubTasks();
-				for (int i = 0; i < numSubTasks; i++) {
-					double ratio = stats.getBackPressureRatio(i);
-
-					gen.writeStartObject();
-					gen.writeNumberField("subtask", i);
-					gen.writeStringField("backpressure-level", getBackPressureLevel(ratio));
-					gen.writeNumberField("ratio", ratio);
-					gen.writeEndObject();
-				}
-				gen.writeEndArray();
-			} else {
-				backPressureStatsTracker.triggerStackTraceSample(jobVertex);
-				gen.writeStringField("status", "deprecated");
-			}
-
-			gen.writeEndObject();
-			gen.close();
-
-			return CompletableFuture.completedFuture(writer.toString());
-		} catch (IOException e) {
-			return FutureUtils.completedExceptionally(e);
-		}
-	}
-
-	/**
-	 * Returns the back pressure level as a String.
-	 *
-	 * @param backPressureRatio Ratio of back pressures samples to total number of samples.
-	 *
-	 * @return Back pressure level ('no', 'low', or 'high')
-	 */
-	static String getBackPressureLevel(double backPressureRatio) {
-		if (backPressureRatio <= 0.10) {
-			return "ok";
-		} else if (backPressureRatio <= 0.5) {
-			return "low";
-		} else {
-			return "high";
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index bd1745c..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java
+++ /dev/null
@@ -1,160 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * A request handler that provides the details of a job vertex, including id, name, parallelism,
- * and the runtime and metrics of all its subtasks.
- */
-public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String JOB_VERTEX_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid";
-
-	private final MetricFetcher fetcher;
-
-	public JobVertexDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
-		super(executionGraphHolder, executor);
-		this.fetcher = fetcher;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_VERTEX_DETAILS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createVertexDetailsJson(jobVertex, params.get("jobid"), fetcher);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not write the vertex details json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the JobVertexDetailsHandler.
-	 */
-	public static class JobVertexDetailsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				String json = createVertexDetailsJson(task, graph.getJobID().toString(), null);
-				String path = JOB_VERTEX_DETAILS_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":vertexid", task.getJobVertexId().toString());
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createVertexDetailsJson(
-			AccessExecutionJobVertex jobVertex,
-			String jobID,
-			@Nullable MetricFetcher fetcher) throws IOException {
-		final long now = System.currentTimeMillis();
-
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-
-		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
-		gen.writeStringField("name", jobVertex.getName());
-		gen.writeNumberField("parallelism", jobVertex.getParallelism());
-		gen.writeNumberField("now", now);
-
-		gen.writeArrayFieldStart("subtasks");
-		int num = 0;
-		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
-			final ExecutionState status = vertex.getExecutionState();
-
-			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
-			String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort();
-
-			long startTime = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
-			if (startTime == 0) {
-				startTime = -1;
-			}
-			long endTime = status.isTerminal() ? vertex.getStateTimestamp(status) : -1;
-			long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1;
-
-			gen.writeStartObject();
-			gen.writeNumberField("subtask", num);
-			gen.writeStringField("status", status.name());
-			gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
-			gen.writeStringField("host", locationString);
-			gen.writeNumberField("start-time", startTime);
-			gen.writeNumberField("end-time", endTime);
-			gen.writeNumberField("duration", duration);
-
-			MutableIOMetrics counts = new MutableIOMetrics();
-
-			counts.addIOMetrics(
-				vertex.getCurrentExecutionAttempt(),
-				fetcher,
-				jobID,
-				jobVertex.getJobVertexId().toString()
-			);
-
-			counts.writeIOMetricsAsJson(gen);
-
-			gen.writeEndObject();
-
-			num++;
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index 0827720..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java
+++ /dev/null
@@ -1,211 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * A request handler that provides the details of a job vertex, including id, name, and the
- * runtime and metrics of all its subtasks aggregated by TaskManager.
- */
-public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String JOB_VERTEX_TASKMANAGERS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/taskmanagers";
-
-	private final MetricFetcher fetcher;
-
-	public JobVertexTaskManagersHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
-		super(executionGraphHolder, executor);
-		this.fetcher = fetcher;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_VERTEX_TASKMANAGERS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createVertexDetailsByTaskManagerJson(jobVertex, params.get("jobid"), fetcher);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create TaskManager json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for JobVertexTaskManagersHandler.
-	 */
-	public static class JobVertexTaskManagersJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				String json = createVertexDetailsByTaskManagerJson(task, graph.getJobID().toString(), null);
-				String path = JOB_VERTEX_TASKMANAGERS_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":vertexid", task.getJobVertexId().toString());
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createVertexDetailsByTaskManagerJson(
-			AccessExecutionJobVertex jobVertex,
-			String jobID,
-			@Nullable MetricFetcher fetcher) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		// Build a map that groups tasks by TaskManager
-		Map<String, List<AccessExecutionVertex>> taskManagerVertices = new HashMap<>();
-
-		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
-			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
-			String taskManager = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort();
-
-			List<AccessExecutionVertex> vertices = taskManagerVertices.get(taskManager);
-
-			if (vertices == null) {
-				vertices = new ArrayList<>();
-				taskManagerVertices.put(taskManager, vertices);
-			}
-
-			vertices.add(vertex);
-		}
-
-		// Build JSON response
-		final long now = System.currentTimeMillis();
-
-		gen.writeStartObject();
-
-		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
-		gen.writeStringField("name", jobVertex.getName());
-		gen.writeNumberField("now", now);
-
-		gen.writeArrayFieldStart("taskmanagers");
-		for (Map.Entry<String, List<AccessExecutionVertex>> entry : taskManagerVertices.entrySet()) {
-			String host = entry.getKey();
-			List<AccessExecutionVertex> taskVertices = entry.getValue();
-
-			int[] tasksPerState = new int[ExecutionState.values().length];
-
-			long startTime = Long.MAX_VALUE;
-			long endTime = 0;
-			boolean allFinished = true;
-
-			MutableIOMetrics counts = new MutableIOMetrics();
-
-			for (AccessExecutionVertex vertex : taskVertices) {
-				final ExecutionState state = vertex.getExecutionState();
-				tasksPerState[state.ordinal()]++;
-
-				// take the earliest start time
-				long started = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
-				if (started > 0) {
-					startTime = Math.min(startTime, started);
-				}
-
-				allFinished &= state.isTerminal();
-				endTime = Math.max(endTime, vertex.getStateTimestamp(state));
-
-				counts.addIOMetrics(
-					vertex.getCurrentExecutionAttempt(),
-					fetcher,
-					jobID,
-					jobVertex.getJobVertexId().toString());
-			}
-
-			long duration;
-			if (startTime < Long.MAX_VALUE) {
-				if (allFinished) {
-					duration = endTime - startTime;
-				}
-				else {
-					endTime = -1L;
-					duration = now - startTime;
-				}
-			}
-			else {
-				startTime = -1L;
-				endTime = -1L;
-				duration = -1L;
-			}
-
-			ExecutionState jobVertexState =
-				ExecutionJobVertex.getAggregateJobVertexState(tasksPerState, taskVertices.size());
-
-			gen.writeStartObject();
-
-			gen.writeStringField("host", host);
-			gen.writeStringField("status", jobVertexState.name());
-
-			gen.writeNumberField("start-time", startTime);
-			gen.writeNumberField("end-time", endTime);
-			gen.writeNumberField("duration", duration);
-
-			counts.writeIOMetricsAsJson(gen);
-
-			gen.writeObjectFieldStart("status-counts");
-			for (ExecutionState state : ExecutionState.values()) {
-				gen.writeNumberField(state.name(), tasksPerState[state.ordinal()]);
-			}
-			gen.writeEndObject();
-
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JsonFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JsonFactory.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JsonFactory.java
deleted file mode 100644
index 4ce0baf..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JsonFactory.java
+++ /dev/null
@@ -1,35 +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.webmonitor.handlers;
-
-/**
- * A holder for the singleton Jackson JSON factory. Since the Jackson's JSON factory object
- * is a heavyweight object that is encouraged to be shared, we use a singleton instance across
- * all request handlers.
- */
-public class JsonFactory {
-
-	/** The singleton Jackson JSON factory. */
-	public static final com.fasterxml.jackson.core.JsonFactory JACKSON_FACTORY =
-			new com.fasterxml.jackson.core.JsonFactory();
-
-	// --------------------------------------------------------------------------------------------
-
-	private JsonFactory() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
deleted file mode 100644
index 8ca785f..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
+++ /dev/null
@@ -1,56 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * Base interface for all request handlers.
- *
- * <p>Most handlers will want to use the {@link AbstractJsonRequestHandler}
- * as a starting point, which produces a valid HTTP response.
- */
-public interface RequestHandler {
-
-	/**
-	 * Core method that handles the request and generates the response. The method needs to
-	 * respond with a full http response, including content-type, content-length, etc.
-	 *
-	 * <p>Exceptions may be throws and will be handled.
-	 *
-	 * @param pathParams The map of REST path parameters, decoded by the router.
-	 * @param queryParams The map of query parameters.
-	 * @param jobManagerGateway to talk to the JobManager.
-	 *
-	 * @return The full http response.
-	 */
-	CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway);
-
-	/**
-	 * Returns an array of REST URL's under which this handler can be registered.
-	 *
-	 * @return array containing REST URL's under which this handler can be registered.
-	 */
-	String[] getPaths();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandlerException.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandlerException.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandlerException.java
deleted file mode 100644
index bb61d16..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandlerException.java
+++ /dev/null
@@ -1,31 +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.webmonitor.handlers;
-
-/**
- * Base class for request handler exceptions.
- */
-public class RequestHandlerException extends Exception {
-
-	private static final long serialVersionUID = 7570352908725875886L;
-
-	public RequestHandlerException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java
deleted file mode 100644
index 301b217..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java
+++ /dev/null
@@ -1,49 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler providing details about a single task execution attempt.
- */
-public class SubtaskCurrentAttemptDetailsHandler extends SubtaskExecutionAttemptDetailsHandler {
-
-	public static final String SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum";
-
-	public SubtaskCurrentAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
-		super(executionGraphHolder, executor, fetcher);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params) {
-		return handleRequest(vertex.getCurrentExecutionAttempt(), params);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java
deleted file mode 100644
index 3c0d1d9..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java
+++ /dev/null
@@ -1,134 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for request handlers whose response depends on a specific job vertex (defined
- * via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
- */
-public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskAttemptRequestHandler {
-
-	private static final String SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators";
-
-	public SubtaskExecutionAttemptAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createAttemptAccumulatorsJson(execAttempt);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create accumulator json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the SubtaskExecutionAttemptAccumulatorsHandler.
-	 */
-	public static class SubtaskExecutionAttemptAccumulatorsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				for (AccessExecutionVertex subtask : task.getTaskVertices()) {
-					String curAttemptJson = createAttemptAccumulatorsJson(subtask.getCurrentExecutionAttempt());
-					String curAttemptPath = SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH
-						.replace(":jobid", graph.getJobID().toString())
-						.replace(":vertexid", task.getJobVertexId().toString())
-						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
-						.replace(":attempt", String.valueOf(subtask.getCurrentExecutionAttempt().getAttemptNumber()));
-
-					archive.add(new ArchivedJson(curAttemptPath, curAttemptJson));
-
-					for (int x = 0; x < subtask.getCurrentExecutionAttempt().getAttemptNumber(); x++) {
-						AccessExecution attempt = subtask.getPriorExecutionAttempt(x);
-						String json = createAttemptAccumulatorsJson(attempt);
-						String path = SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH
-							.replace(":jobid", graph.getJobID().toString())
-							.replace(":vertexid", task.getJobVertexId().toString())
-							.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
-							.replace(":attempt", String.valueOf(attempt.getAttemptNumber()));
-						archive.add(new ArchivedJson(path, json));
-					}
-				}
-			}
-			return archive;
-		}
-	}
-
-	public static String createAttemptAccumulatorsJson(AccessExecution execAttempt) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		final StringifiedAccumulatorResult[] accs = execAttempt.getUserAccumulatorsStringified();
-
-		gen.writeStartObject();
-
-		gen.writeNumberField("subtask", execAttempt.getParallelSubtaskIndex());
-		gen.writeNumberField("attempt", execAttempt.getAttemptNumber());
-		gen.writeStringField("id", execAttempt.getAttemptId().toString());
-
-		gen.writeArrayFieldStart("user-accumulators");
-		for (StringifiedAccumulatorResult acc : accs) {
-			gen.writeStartObject();
-			gen.writeStringField("name", acc.getName());
-			gen.writeStringField("type", acc.getType());
-			gen.writeStringField("value", acc.getValue());
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index ad836df..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java
+++ /dev/null
@@ -1,167 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.runtime.webmonitor.handlers.SubtaskCurrentAttemptDetailsHandler.SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH;
-
-/**
- * Request handler providing details about a single task execution attempt.
- */
-public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemptRequestHandler {
-
-	private static final String SUBTASK_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt";
-
-	private final MetricFetcher fetcher;
-
-	public SubtaskExecutionAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
-		super(executionGraphHolder, executor);
-		this.fetcher = fetcher;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{SUBTASK_ATTEMPT_DETAILS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createAttemptDetailsJson(execAttempt, params.get("jobid"), params.get("vertexid"), fetcher);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create attempt details json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the SubtaskExecutionAttemptDetailsHandler.
-	 */
-	public static class SubtaskExecutionAttemptDetailsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				for (AccessExecutionVertex subtask : task.getTaskVertices()) {
-					String curAttemptJson = createAttemptDetailsJson(subtask.getCurrentExecutionAttempt(), graph.getJobID().toString(), task.getJobVertexId().toString(), null);
-					String curAttemptPath1 = SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH
-						.replace(":jobid", graph.getJobID().toString())
-						.replace(":vertexid", task.getJobVertexId().toString())
-						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()));
-					String curAttemptPath2 = SUBTASK_ATTEMPT_DETAILS_REST_PATH
-						.replace(":jobid", graph.getJobID().toString())
-						.replace(":vertexid", task.getJobVertexId().toString())
-						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
-						.replace(":attempt", String.valueOf(subtask.getCurrentExecutionAttempt().getAttemptNumber()));
-
-					archive.add(new ArchivedJson(curAttemptPath1, curAttemptJson));
-					archive.add(new ArchivedJson(curAttemptPath2, curAttemptJson));
-
-					for (int x = 0; x < subtask.getCurrentExecutionAttempt().getAttemptNumber(); x++) {
-						AccessExecution attempt = subtask.getPriorExecutionAttempt(x);
-						String json = createAttemptDetailsJson(attempt, graph.getJobID().toString(), task.getJobVertexId().toString(), null);
-						String path = SUBTASK_ATTEMPT_DETAILS_REST_PATH
-							.replace(":jobid", graph.getJobID().toString())
-							.replace(":vertexid", task.getJobVertexId().toString())
-							.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
-							.replace(":attempt", String.valueOf(attempt.getAttemptNumber()));
-						archive.add(new ArchivedJson(path, json));
-					}
-				}
-			}
-			return archive;
-		}
-	}
-
-	public static String createAttemptDetailsJson(
-			AccessExecution execAttempt,
-			String jobID,
-			String vertexID,
-			@Nullable MetricFetcher fetcher) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		final ExecutionState status = execAttempt.getState();
-		final long now = System.currentTimeMillis();
-
-		TaskManagerLocation location = execAttempt.getAssignedResourceLocation();
-		String locationString = location == null ? "(unassigned)" : location.getHostname();
-
-		long startTime = execAttempt.getStateTimestamp(ExecutionState.DEPLOYING);
-		if (startTime == 0) {
-			startTime = -1;
-		}
-		long endTime = status.isTerminal() ? execAttempt.getStateTimestamp(status) : -1;
-		long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1;
-
-		gen.writeStartObject();
-		gen.writeNumberField("subtask", execAttempt.getParallelSubtaskIndex());
-		gen.writeStringField("status", status.name());
-		gen.writeNumberField("attempt", execAttempt.getAttemptNumber());
-		gen.writeStringField("host", locationString);
-		gen.writeNumberField("start-time", startTime);
-		gen.writeNumberField("end-time", endTime);
-		gen.writeNumberField("duration", duration);
-
-		MutableIOMetrics counts = new MutableIOMetrics();
-
-		counts.addIOMetrics(
-			execAttempt,
-			fetcher,
-			jobID,
-			vertexID
-		);
-
-		counts.writeIOMetricsAsJson(gen);
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index 8142548..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java
+++ /dev/null
@@ -1,131 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the accumulators for all subtasks of job vertex.
- */
-public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String SUBTASKS_ALL_ACCUMULATORS_REST_PATH = 	"/jobs/:jobid/vertices/:vertexid/subtasks/accumulators";
-
-	public SubtasksAllAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{SUBTASKS_ALL_ACCUMULATORS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createSubtasksAccumulatorsJson(jobVertex);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create subtasks accumulator json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the SubtasksAllAccumulatorsHandler.
-	 */
-	public static class SubtasksAllAccumulatorsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				String json = createSubtasksAccumulatorsJson(task);
-				String path = SUBTASKS_ALL_ACCUMULATORS_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":vertexid", task.getJobVertexId().toString());
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createSubtasksAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
-		gen.writeNumberField("parallelism", jobVertex.getParallelism());
-
-		gen.writeArrayFieldStart("subtasks");
-
-		int num = 0;
-		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
-
-			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
-			String locationString = location == null ? "(unassigned)" : location.getHostname();
-
-			gen.writeStartObject();
-
-			gen.writeNumberField("subtask", num++);
-			gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
-			gen.writeStringField("host", locationString);
-
-			StringifiedAccumulatorResult[] accs = vertex.getCurrentExecutionAttempt().getUserAccumulatorsStringified();
-			gen.writeArrayFieldStart("user-accumulators");
-			for (StringifiedAccumulatorResult acc : accs) {
-				gen.writeStartObject();
-				gen.writeStringField("name", acc.getName());
-				gen.writeStringField("type", acc.getType());
-				gen.writeStringField("value", acc.getValue());
-				gen.writeEndObject();
-			}
-			gen.writeEndArray();
-
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index d766206..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java
+++ /dev/null
@@ -1,141 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the state transition timestamps for all subtasks, plus their
- * location and duration.
- */
-public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
-
-	private static final String SUBTASK_TIMES_REST_PATH = 	"/jobs/:jobid/vertices/:vertexid/subtasktimes";
-
-	public SubtasksTimesHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{SUBTASK_TIMES_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createSubtaskTimesJson(jobVertex);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not write subtask time json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the SubtasksTimesHandler.
-	 */
-	public static class SubtasksTimesJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
-				String json = createSubtaskTimesJson(task);
-				String path = SUBTASK_TIMES_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":vertexid", task.getJobVertexId().toString());
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createSubtaskTimesJson(AccessExecutionJobVertex jobVertex) throws IOException {
-		final long now = System.currentTimeMillis();
-
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-
-		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
-		gen.writeStringField("name", jobVertex.getName());
-		gen.writeNumberField("now", now);
-
-		gen.writeArrayFieldStart("subtasks");
-
-		int num = 0;
-		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
-
-			long[] timestamps = vertex.getCurrentExecutionAttempt().getStateTimestamps();
-			ExecutionState status = vertex.getExecutionState();
-
-			long scheduledTime = timestamps[ExecutionState.SCHEDULED.ordinal()];
-
-			long start = scheduledTime > 0 ? scheduledTime : -1;
-			long end = status.isTerminal() ? timestamps[status.ordinal()] : now;
-			long duration = start >= 0 ? end - start : -1L;
-
-			gen.writeStartObject();
-			gen.writeNumberField("subtask", num++);
-
-			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
-			String locationString = location == null ? "(unassigned)" : location.getHostname();
-			gen.writeStringField("host", locationString);
-
-			gen.writeNumberField("duration", duration);
-
-			gen.writeObjectFieldStart("timestamps");
-			for (ExecutionState state : ExecutionState.values()) {
-				gen.writeNumberField(state.name(), timestamps[state.ordinal()]);
-			}
-			gen.writeEndObject();
-
-			gen.writeEndObject();
-		}
-
-		gen.writeEndArray();
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
deleted file mode 100644
index d53d2b1..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
+++ /dev/null
@@ -1,335 +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.webmonitor.handlers;
-
-/*****************************************************************************
- * This code is based on the "HttpStaticFileServerHandler" from the
- * Netty project's HTTP server example.
- *
- * See http://netty.io and
- * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
- *****************************************************************************/
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.blob.BlobCache;
-import org.apache.flink.runtime.blob.BlobKey;
-import org.apache.flink.runtime.blob.BlobView;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.rest.handler.RedirectHandler;
-import org.apache.flink.runtime.webmonitor.WebHandler;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.StringUtils;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
-import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
-import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
-import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.net.InetSocketAddress;
-import java.nio.channels.FileChannel;
-import java.util.HashMap;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Request handler that returns the TaskManager log/out files.
- *
- * <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
- * example.</p>
- */
-@ChannelHandler.Sharable
-public class TaskManagerLogHandler extends RedirectHandler<JobManagerGateway> implements WebHandler {
-	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerLogHandler.class);
-
-	private static final String TASKMANAGER_LOG_REST_PATH = "/taskmanagers/:taskmanagerid/log";
-	private static final String TASKMANAGER_OUT_REST_PATH = "/taskmanagers/:taskmanagerid/stdout";
-
-	/** Keep track of last transmitted log, to clean up old ones. */
-	private final HashMap<String, BlobKey> lastSubmittedLog = new HashMap<>();
-	private final HashMap<String, BlobKey> lastSubmittedStdout = new HashMap<>();
-
-	/** Keep track of request status, prevents multiple log requests for a single TM running concurrently. */
-	private final ConcurrentHashMap<String, Boolean> lastRequestPending = new ConcurrentHashMap<>();
-	private final Configuration config;
-
-	/** Future of the blob cache. */
-	private CompletableFuture<BlobCache> cache;
-
-	/** Indicates which log file should be displayed. */
-	private FileMode fileMode;
-
-	private final Executor executor;
-
-	private final BlobView blobView;
-
-	/** Used to control whether this handler serves the .log or .out file. */
-	public enum FileMode {
-		LOG,
-		STDOUT
-	}
-
-	public TaskManagerLogHandler(
-		GatewayRetriever<JobManagerGateway> retriever,
-		Executor executor,
-		CompletableFuture<String> localJobManagerAddressPromise,
-		Time timeout,
-		FileMode fileMode,
-		Configuration config,
-		BlobView blobView) {
-		super(localJobManagerAddressPromise, retriever, timeout);
-
-		this.executor = checkNotNull(executor);
-		this.config = config;
-		this.fileMode = fileMode;
-
-		this.blobView = Preconditions.checkNotNull(blobView, "blobView");
-	}
-
-	@Override
-	public String[] getPaths() {
-		switch (fileMode) {
-			case LOG:
-				return new String[]{TASKMANAGER_LOG_REST_PATH};
-			case STDOUT:
-			default:
-				return new String[]{TASKMANAGER_OUT_REST_PATH};
-		}
-	}
-
-	/**
-	 * Response when running with leading JobManager.
-	 */
-	@Override
-	protected void respondAsLeader(final ChannelHandlerContext ctx, final Routed routed, final JobManagerGateway jobManagerGateway) {
-		if (cache == null) {
-			CompletableFuture<Integer> blobPortFuture = jobManagerGateway.requestBlobServerPort(timeout);
-			cache = blobPortFuture.thenApplyAsync(
-				(Integer port) -> {
-					try {
-						return new BlobCache(new InetSocketAddress(jobManagerGateway.getHostname(), port), config, blobView);
-					} catch (IOException e) {
-						throw new FlinkFutureException("Could not create BlobCache.", e);
-					}
-				},
-				executor);
-		}
-
-		final String taskManagerID = routed.pathParams().get(TaskManagersHandler.TASK_MANAGER_ID_KEY);
-		final HttpRequest request = routed.request();
-
-		//fetch TaskManager logs if no other process is currently doing it
-		if (lastRequestPending.putIfAbsent(taskManagerID, true) == null) {
-			try {
-				InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(taskManagerID));
-				CompletableFuture<Optional<Instance>> taskManagerFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout);
-
-				CompletableFuture<BlobKey> blobKeyFuture = taskManagerFuture.thenCompose(
-					(Optional<Instance> optTMInstance) -> {
-						Instance taskManagerInstance = optTMInstance.orElseThrow(
-							() -> new FlinkFutureException("Could not find instance with " + instanceID + '.'));
-						switch (fileMode) {
-							case LOG:
-								return taskManagerInstance.getTaskManagerGateway().requestTaskManagerLog(timeout);
-							case STDOUT:
-							default:
-								return taskManagerInstance.getTaskManagerGateway().requestTaskManagerStdout(timeout);
-						}
-					}
-				);
-
-				CompletableFuture<String> logPathFuture = blobKeyFuture
-					.thenCombineAsync(
-						cache,
-						(blobKey, blobCache) -> {
-							//delete previous log file, if it is different than the current one
-							HashMap<String, BlobKey> lastSubmittedFile = fileMode == FileMode.LOG ? lastSubmittedLog : lastSubmittedStdout;
-							if (lastSubmittedFile.containsKey(taskManagerID)) {
-								if (!Objects.equals(blobKey, lastSubmittedFile.get(taskManagerID))) {
-									try {
-										blobCache.deleteGlobal(lastSubmittedFile.get(taskManagerID));
-									} catch (IOException e) {
-										throw new FlinkFutureException("Could not delete file for " + taskManagerID + '.', e);
-									}
-									lastSubmittedFile.put(taskManagerID, blobKey);
-								}
-							} else {
-								lastSubmittedFile.put(taskManagerID, blobKey);
-							}
-							try {
-								return blobCache.getFile(blobKey).getAbsolutePath();
-							} catch (IOException e) {
-								throw new FlinkFutureException("Could not retrieve blob for " + blobKey + '.', e);
-							}
-						},
-						executor);
-
-				logPathFuture.exceptionally(
-					failure -> {
-						display(ctx, request, "Fetching TaskManager log failed.");
-						LOG.error("Fetching TaskManager log failed.", failure);
-						lastRequestPending.remove(taskManagerID);
-
-						return null;
-					});
-
-				logPathFuture.thenAccept(
-					filePath -> {
-						File file = new File(filePath);
-						final RandomAccessFile raf;
-						try {
-							raf = new RandomAccessFile(file, "r");
-						} catch (FileNotFoundException e) {
-							display(ctx, request, "Displaying TaskManager log failed.");
-							LOG.error("Displaying TaskManager log failed.", e);
-
-							return;
-						}
-						long fileLength;
-						try {
-							fileLength = raf.length();
-						} catch (IOException ioe) {
-							display(ctx, request, "Displaying TaskManager log failed.");
-							LOG.error("Displaying TaskManager log failed.", ioe);
-							try {
-								raf.close();
-							} catch (IOException e) {
-								LOG.error("Could not close random access file.", e);
-							}
-
-							return;
-						}
-						final FileChannel fc = raf.getChannel();
-
-						HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-						response.headers().set(CONTENT_TYPE, "text/plain");
-
-						if (HttpHeaders.isKeepAlive(request)) {
-							response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
-						}
-						HttpHeaders.setContentLength(response, fileLength);
-
-						// write the initial line and the header.
-						ctx.write(response);
-
-						// write the content.
-						ChannelFuture lastContentFuture;
-						final GenericFutureListener<Future<? super Void>> completionListener = future -> {
-							lastRequestPending.remove(taskManagerID);
-							fc.close();
-							raf.close();
-						};
-						if (ctx.pipeline().get(SslHandler.class) == null) {
-							ctx.write(
-								new DefaultFileRegion(fc, 0, fileLength), ctx.newProgressivePromise())
-									.addListener(completionListener);
-							lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
-
-						} else {
-							try {
-								lastContentFuture = ctx.writeAndFlush(
-									new HttpChunkedInput(new ChunkedFile(raf, 0, fileLength, 8192)),
-									ctx.newProgressivePromise())
-									.addListener(completionListener);
-							} catch (IOException e) {
-								display(ctx, request, "Displaying TaskManager log failed.");
-								LOG.warn("Could not write http data.", e);
-
-								return;
-							}
-							// HttpChunkedInput will write the end marker (LastHttpContent) for us.
-						}
-
-						// close the connection, if no keep-alive is needed
-						if (!HttpHeaders.isKeepAlive(request)) {
-							lastContentFuture.addListener(ChannelFutureListener.CLOSE);
-						}
-					});
-			} catch (Exception e) {
-				display(ctx, request, "Error: " + e.getMessage());
-				LOG.error("Fetching TaskManager log failed.", e);
-				lastRequestPending.remove(taskManagerID);
-			}
-		} else {
-			display(ctx, request, "loading...");
-		}
-	}
-
-	private void display(ChannelHandlerContext ctx, HttpRequest request, String message) {
-		HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-		response.headers().set(CONTENT_TYPE, "text/plain");
-
-		if (HttpHeaders.isKeepAlive(request)) {
-			response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
-		}
-
-		byte[] buf = message.getBytes(ConfigConstants.DEFAULT_CHARSET);
-
-		ByteBuf b = Unpooled.copiedBuffer(buf);
-
-		HttpHeaders.setContentLength(response, buf.length);
-
-		// write the initial line and the header.
-		ctx.write(response);
-
-		ctx.write(b);
-
-		ChannelFuture lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
-
-		// close the connection, if no keep-alive is needed
-		if (!HttpHeaders.isKeepAlive(request)) {
-			lastContentFuture.addListener(ChannelFutureListener.CLOSE);
-		}
-	}
-}


[02/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
new file mode 100644
index 0000000..4be7840
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandlerTest.java
@@ -0,0 +1,432 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStatsSummary;
+import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
+import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
+import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the CheckpointStatsHandler.
+ */
+public class CheckpointStatsHandlerTest {
+
+	@Test
+	public void testArchiver() throws IOException {
+		JsonArchivist archivist = new CheckpointStatsDetailsHandler.CheckpointStatsDetailsJsonArchivist();
+		TestCheckpointStats testCheckpointStats = createTestCheckpointStats();
+		when(testCheckpointStats.graph.getJobID()).thenReturn(new JobID());
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(testCheckpointStats.graph);
+		Assert.assertEquals(3, archives.size());
+
+		ObjectMapper mapper = new ObjectMapper();
+
+		Iterator<ArchivedJson> iterator = archives.iterator();
+		ArchivedJson archive1 = iterator.next();
+		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.inProgress.getCheckpointId(), archive1.getPath());
+		compareInProgressCheckpoint(testCheckpointStats.inProgress, mapper.readTree(archive1.getJson()));
+
+		ArchivedJson archive2 = iterator.next();
+		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.completedSavepoint.getCheckpointId(), archive2.getPath());
+		compareCompletedSavepoint(testCheckpointStats.completedSavepoint, mapper.readTree(archive2.getJson()));
+
+		ArchivedJson archive3 = iterator.next();
+		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.failed.getCheckpointId(), archive3.getPath());
+		compareFailedCheckpoint(testCheckpointStats.failed, mapper.readTree(archive3.getJson()));
+	}
+
+	@Test
+	public void testGetPaths() {
+		CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/checkpoints", paths[0]);
+	}
+
+	/**
+	 * Tests a complete checkpoint stats snapshot.
+	 */
+	@Test
+	public void testCheckpointStatsRequest() throws Exception {
+		TestCheckpointStats testCheckpointStats = createTestCheckpointStats();
+
+		CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String json = handler.handleRequest(testCheckpointStats.graph, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(json);
+
+		compareCheckpointStats(testCheckpointStats, rootNode);
+	}
+
+	private static TestCheckpointStats createTestCheckpointStats() {
+		// Counts
+		CheckpointStatsCounts counts = mock(CheckpointStatsCounts.class);
+		when(counts.getNumberOfRestoredCheckpoints()).thenReturn(123123123L);
+		when(counts.getTotalNumberOfCheckpoints()).thenReturn(12981231203L);
+		when(counts.getNumberOfInProgressCheckpoints()).thenReturn(191919);
+		when(counts.getNumberOfCompletedCheckpoints()).thenReturn(882828200L);
+		when(counts.getNumberOfFailedCheckpoints()).thenReturn(99171510L);
+
+		// Summary
+		CompletedCheckpointStatsSummary summary = mock(CompletedCheckpointStatsSummary.class);
+
+		MinMaxAvgStats stateSizeSummary = mock(MinMaxAvgStats.class);
+		when(stateSizeSummary.getMinimum()).thenReturn(81238123L);
+		when(stateSizeSummary.getMaximum()).thenReturn(19919191999L);
+		when(stateSizeSummary.getAverage()).thenReturn(1133L);
+
+		MinMaxAvgStats durationSummary = mock(MinMaxAvgStats.class);
+		when(durationSummary.getMinimum()).thenReturn(1182L);
+		when(durationSummary.getMaximum()).thenReturn(88654L);
+		when(durationSummary.getAverage()).thenReturn(171L);
+
+		MinMaxAvgStats alignmentBufferedSummary = mock(MinMaxAvgStats.class);
+		when(alignmentBufferedSummary.getMinimum()).thenReturn(81818181899L);
+		when(alignmentBufferedSummary.getMaximum()).thenReturn(89999911118654L);
+		when(alignmentBufferedSummary.getAverage()).thenReturn(11203131L);
+
+		when(summary.getStateSizeStats()).thenReturn(stateSizeSummary);
+		when(summary.getEndToEndDurationStats()).thenReturn(durationSummary);
+		when(summary.getAlignmentBufferedStats()).thenReturn(alignmentBufferedSummary);
+
+		// Latest
+		CompletedCheckpointStats latestCompleted = mock(CompletedCheckpointStats.class);
+		when(latestCompleted.getCheckpointId()).thenReturn(1992139L);
+		when(latestCompleted.getTriggerTimestamp()).thenReturn(1919191900L);
+		when(latestCompleted.getLatestAckTimestamp()).thenReturn(1977791901L);
+		when(latestCompleted.getStateSize()).thenReturn(111939272822L);
+		when(latestCompleted.getEndToEndDuration()).thenReturn(121191L);
+		when(latestCompleted.getAlignmentBuffered()).thenReturn(1L);
+		when(latestCompleted.getExternalPath()).thenReturn("latest-completed-external-path");
+
+		CompletedCheckpointStats latestSavepoint = mock(CompletedCheckpointStats.class);
+		when(latestSavepoint.getCheckpointId()).thenReturn(1992140L);
+		when(latestSavepoint.getTriggerTimestamp()).thenReturn(1919191900L);
+		when(latestSavepoint.getLatestAckTimestamp()).thenReturn(1977791901L);
+		when(latestSavepoint.getStateSize()).thenReturn(111939272822L);
+		when(latestSavepoint.getEndToEndDuration()).thenReturn(121191L);
+		when(latestCompleted.getAlignmentBuffered()).thenReturn(182813L);
+		when(latestSavepoint.getExternalPath()).thenReturn("savepoint-external-path");
+
+		FailedCheckpointStats latestFailed = mock(FailedCheckpointStats.class);
+		when(latestFailed.getCheckpointId()).thenReturn(1112L);
+		when(latestFailed.getTriggerTimestamp()).thenReturn(12828L);
+		when(latestFailed.getLatestAckTimestamp()).thenReturn(1901L);
+		when(latestFailed.getFailureTimestamp()).thenReturn(11999976L);
+		when(latestFailed.getStateSize()).thenReturn(111L);
+		when(latestFailed.getEndToEndDuration()).thenReturn(12L);
+		when(latestFailed.getAlignmentBuffered()).thenReturn(2L);
+		when(latestFailed.getFailureMessage()).thenReturn("expected cause");
+
+		RestoredCheckpointStats latestRestored = mock(RestoredCheckpointStats.class);
+		when(latestRestored.getCheckpointId()).thenReturn(1199L);
+		when(latestRestored.getRestoreTimestamp()).thenReturn(434242L);
+		when(latestRestored.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(latestRestored.getExternalPath()).thenReturn("restored savepoint path");
+
+		// History
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		List<AbstractCheckpointStats> checkpoints = new ArrayList<>();
+
+		PendingCheckpointStats inProgress = mock(PendingCheckpointStats.class);
+		when(inProgress.getCheckpointId()).thenReturn(1992141L);
+		when(inProgress.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
+		when(inProgress.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(inProgress.getTriggerTimestamp()).thenReturn(1919191900L);
+		when(inProgress.getLatestAckTimestamp()).thenReturn(1977791901L);
+		when(inProgress.getStateSize()).thenReturn(111939272822L);
+		when(inProgress.getEndToEndDuration()).thenReturn(121191L);
+		when(inProgress.getAlignmentBuffered()).thenReturn(1L);
+		when(inProgress.getNumberOfSubtasks()).thenReturn(501);
+		when(inProgress.getNumberOfAcknowledgedSubtasks()).thenReturn(101);
+
+		CompletedCheckpointStats completedSavepoint = mock(CompletedCheckpointStats.class);
+		when(completedSavepoint.getCheckpointId()).thenReturn(1322139L);
+		when(completedSavepoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
+		when(completedSavepoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
+		when(completedSavepoint.getTriggerTimestamp()).thenReturn(191900L);
+		when(completedSavepoint.getLatestAckTimestamp()).thenReturn(197791901L);
+		when(completedSavepoint.getStateSize()).thenReturn(1119822L);
+		when(completedSavepoint.getEndToEndDuration()).thenReturn(12191L);
+		when(completedSavepoint.getAlignmentBuffered()).thenReturn(111L);
+		when(completedSavepoint.getNumberOfSubtasks()).thenReturn(33501);
+		when(completedSavepoint.getNumberOfAcknowledgedSubtasks()).thenReturn(211);
+		when(completedSavepoint.isDiscarded()).thenReturn(true);
+		when(completedSavepoint.getExternalPath()).thenReturn("completed-external-path");
+
+		FailedCheckpointStats failed = mock(FailedCheckpointStats.class);
+		when(failed.getCheckpointId()).thenReturn(110719L);
+		when(failed.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
+		when(failed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
+		when(failed.getTriggerTimestamp()).thenReturn(191900L);
+		when(failed.getLatestAckTimestamp()).thenReturn(197791901L);
+		when(failed.getStateSize()).thenReturn(1119822L);
+		when(failed.getEndToEndDuration()).thenReturn(12191L);
+		when(failed.getAlignmentBuffered()).thenReturn(111L);
+		when(failed.getNumberOfSubtasks()).thenReturn(33501);
+		when(failed.getNumberOfAcknowledgedSubtasks()).thenReturn(1);
+		when(failed.getFailureTimestamp()).thenReturn(119230L);
+		when(failed.getFailureMessage()).thenReturn("failure message");
+
+		checkpoints.add(inProgress);
+		checkpoints.add(completedSavepoint);
+		checkpoints.add(failed);
+		when(history.getCheckpoints()).thenReturn(checkpoints);
+		when(history.getLatestCompletedCheckpoint()).thenReturn(latestCompleted);
+		when(history.getLatestSavepoint()).thenReturn(latestSavepoint);
+		when(history.getLatestFailedCheckpoint()).thenReturn(latestFailed);
+
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getCounts()).thenReturn(counts);
+		when(snapshot.getSummaryStats()).thenReturn(summary);
+		when(snapshot.getHistory()).thenReturn(history);
+		when(snapshot.getLatestRestoredCheckpoint()).thenReturn(latestRestored);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		return new TestCheckpointStats(
+			graph, counts, stateSizeSummary, durationSummary, alignmentBufferedSummary, summary,
+			latestCompleted, latestSavepoint, latestFailed, latestRestored, inProgress,
+			completedSavepoint, failed, history, snapshot
+		);
+	}
+
+	private static void compareCheckpointStats(TestCheckpointStats checkpointStats, JsonNode rootNode) {
+		CheckpointStatsCounts counts = checkpointStats.counts;
+		JsonNode countNode = rootNode.get("counts");
+		assertEquals(counts.getNumberOfRestoredCheckpoints(), countNode.get("restored").asLong());
+		assertEquals(counts.getTotalNumberOfCheckpoints(), countNode.get("total").asLong());
+		assertEquals(counts.getNumberOfInProgressCheckpoints(), countNode.get("in_progress").asLong());
+		assertEquals(counts.getNumberOfCompletedCheckpoints(), countNode.get("completed").asLong());
+		assertEquals(counts.getNumberOfFailedCheckpoints(), countNode.get("failed").asLong());
+
+		MinMaxAvgStats stateSizeSummary = checkpointStats.stateSizeSummary;
+		JsonNode summaryNode = rootNode.get("summary");
+		JsonNode sizeSummaryNode = summaryNode.get("state_size");
+		assertEquals(stateSizeSummary.getMinimum(), sizeSummaryNode.get("min").asLong());
+		assertEquals(stateSizeSummary.getMaximum(), sizeSummaryNode.get("max").asLong());
+		assertEquals(stateSizeSummary.getAverage(), sizeSummaryNode.get("avg").asLong());
+
+		MinMaxAvgStats durationSummary = checkpointStats.durationSummary;
+		JsonNode durationSummaryNode = summaryNode.get("end_to_end_duration");
+		assertEquals(durationSummary.getMinimum(), durationSummaryNode.get("min").asLong());
+		assertEquals(durationSummary.getMaximum(), durationSummaryNode.get("max").asLong());
+		assertEquals(durationSummary.getAverage(), durationSummaryNode.get("avg").asLong());
+
+		MinMaxAvgStats alignmentBufferedSummary = checkpointStats.alignmentBufferedSummary;
+		JsonNode alignmentBufferedNode = summaryNode.get("alignment_buffered");
+		assertEquals(alignmentBufferedSummary.getMinimum(), alignmentBufferedNode.get("min").asLong());
+		assertEquals(alignmentBufferedSummary.getMaximum(), alignmentBufferedNode.get("max").asLong());
+		assertEquals(alignmentBufferedSummary.getAverage(), alignmentBufferedNode.get("avg").asLong());
+
+		CompletedCheckpointStats latestCompleted = checkpointStats.latestCompleted;
+		JsonNode latestNode = rootNode.get("latest");
+		JsonNode latestCheckpointNode = latestNode.get("completed");
+		assertEquals(latestCompleted.getCheckpointId(), latestCheckpointNode.get("id").asLong());
+		assertEquals(latestCompleted.getTriggerTimestamp(), latestCheckpointNode.get("trigger_timestamp").asLong());
+		assertEquals(latestCompleted.getLatestAckTimestamp(), latestCheckpointNode.get("latest_ack_timestamp").asLong());
+		assertEquals(latestCompleted.getStateSize(), latestCheckpointNode.get("state_size").asLong());
+		assertEquals(latestCompleted.getEndToEndDuration(), latestCheckpointNode.get("end_to_end_duration").asLong());
+		assertEquals(latestCompleted.getAlignmentBuffered(), latestCheckpointNode.get("alignment_buffered").asLong());
+		assertEquals(latestCompleted.getExternalPath(), latestCheckpointNode.get("external_path").asText());
+
+		CompletedCheckpointStats latestSavepoint = checkpointStats.latestSavepoint;
+		JsonNode latestSavepointNode = latestNode.get("savepoint");
+		assertEquals(latestSavepoint.getCheckpointId(), latestSavepointNode.get("id").asLong());
+		assertEquals(latestSavepoint.getTriggerTimestamp(), latestSavepointNode.get("trigger_timestamp").asLong());
+		assertEquals(latestSavepoint.getLatestAckTimestamp(), latestSavepointNode.get("latest_ack_timestamp").asLong());
+		assertEquals(latestSavepoint.getStateSize(), latestSavepointNode.get("state_size").asLong());
+		assertEquals(latestSavepoint.getEndToEndDuration(), latestSavepointNode.get("end_to_end_duration").asLong());
+		assertEquals(latestSavepoint.getAlignmentBuffered(), latestSavepointNode.get("alignment_buffered").asLong());
+		assertEquals(latestSavepoint.getExternalPath(), latestSavepointNode.get("external_path").asText());
+
+		FailedCheckpointStats latestFailed = checkpointStats.latestFailed;
+		JsonNode latestFailedNode = latestNode.get("failed");
+		assertEquals(latestFailed.getCheckpointId(), latestFailedNode.get("id").asLong());
+		assertEquals(latestFailed.getTriggerTimestamp(), latestFailedNode.get("trigger_timestamp").asLong());
+		assertEquals(latestFailed.getLatestAckTimestamp(), latestFailedNode.get("latest_ack_timestamp").asLong());
+		assertEquals(latestFailed.getStateSize(), latestFailedNode.get("state_size").asLong());
+		assertEquals(latestFailed.getEndToEndDuration(), latestFailedNode.get("end_to_end_duration").asLong());
+		assertEquals(latestFailed.getAlignmentBuffered(), latestFailedNode.get("alignment_buffered").asLong());
+		assertEquals(latestFailed.getFailureTimestamp(), latestFailedNode.get("failure_timestamp").asLong());
+		assertEquals(latestFailed.getFailureMessage(), latestFailedNode.get("failure_message").asText());
+
+		RestoredCheckpointStats latestRestored = checkpointStats.latestRestored;
+		JsonNode latestRestoredNode = latestNode.get("restored");
+		assertEquals(latestRestored.getCheckpointId(), latestRestoredNode.get("id").asLong());
+		assertEquals(latestRestored.getRestoreTimestamp(), latestRestoredNode.get("restore_timestamp").asLong());
+		assertEquals(latestRestored.getProperties().isSavepoint(), latestRestoredNode.get("is_savepoint").asBoolean());
+		assertEquals(latestRestored.getExternalPath(), latestRestoredNode.get("external_path").asText());
+
+		JsonNode historyNode = rootNode.get("history");
+		Iterator<JsonNode> it = historyNode.iterator();
+
+		assertTrue(it.hasNext());
+		JsonNode inProgressNode = it.next();
+
+		PendingCheckpointStats inProgress = checkpointStats.inProgress;
+		compareInProgressCheckpoint(inProgress, inProgressNode);
+
+		assertTrue(it.hasNext());
+		JsonNode completedSavepointNode = it.next();
+
+		CompletedCheckpointStats completedSavepoint = checkpointStats.completedSavepoint;
+		compareCompletedSavepoint(completedSavepoint, completedSavepointNode);
+
+		assertTrue(it.hasNext());
+		JsonNode failedNode = it.next();
+
+		FailedCheckpointStats failed = checkpointStats.failed;
+		compareFailedCheckpoint(failed, failedNode);
+
+		assertFalse(it.hasNext());
+	}
+
+	private static void compareInProgressCheckpoint(PendingCheckpointStats inProgress, JsonNode inProgressNode) {
+		assertEquals(inProgress.getCheckpointId(), inProgressNode.get("id").asLong());
+		assertEquals(inProgress.getStatus().toString(), inProgressNode.get("status").asText());
+		assertEquals(inProgress.getProperties().isSavepoint(), inProgressNode.get("is_savepoint").asBoolean());
+		assertEquals(inProgress.getTriggerTimestamp(), inProgressNode.get("trigger_timestamp").asLong());
+		assertEquals(inProgress.getLatestAckTimestamp(), inProgressNode.get("latest_ack_timestamp").asLong());
+		assertEquals(inProgress.getStateSize(), inProgressNode.get("state_size").asLong());
+		assertEquals(inProgress.getEndToEndDuration(), inProgressNode.get("end_to_end_duration").asLong());
+		assertEquals(inProgress.getAlignmentBuffered(), inProgressNode.get("alignment_buffered").asLong());
+		assertEquals(inProgress.getNumberOfSubtasks(), inProgressNode.get("num_subtasks").asInt());
+		assertEquals(inProgress.getNumberOfAcknowledgedSubtasks(), inProgressNode.get("num_acknowledged_subtasks").asInt());
+	}
+
+	private static void compareCompletedSavepoint(CompletedCheckpointStats completedSavepoint, JsonNode completedSavepointNode) {
+		assertEquals(completedSavepoint.getCheckpointId(), completedSavepointNode.get("id").asLong());
+		assertEquals(completedSavepoint.getStatus().toString(), completedSavepointNode.get("status").asText());
+		assertEquals(completedSavepoint.getProperties().isSavepoint(), completedSavepointNode.get("is_savepoint").asBoolean());
+		assertEquals(completedSavepoint.getTriggerTimestamp(), completedSavepointNode.get("trigger_timestamp").asLong());
+		assertEquals(completedSavepoint.getLatestAckTimestamp(), completedSavepointNode.get("latest_ack_timestamp").asLong());
+		assertEquals(completedSavepoint.getStateSize(), completedSavepointNode.get("state_size").asLong());
+		assertEquals(completedSavepoint.getEndToEndDuration(), completedSavepointNode.get("end_to_end_duration").asLong());
+		assertEquals(completedSavepoint.getAlignmentBuffered(), completedSavepointNode.get("alignment_buffered").asLong());
+		assertEquals(completedSavepoint.getNumberOfSubtasks(), completedSavepointNode.get("num_subtasks").asInt());
+		assertEquals(completedSavepoint.getNumberOfAcknowledgedSubtasks(), completedSavepointNode.get("num_acknowledged_subtasks").asInt());
+
+		assertEquals(completedSavepoint.getExternalPath(), completedSavepointNode.get("external_path").asText());
+		assertEquals(completedSavepoint.isDiscarded(), completedSavepointNode.get("discarded").asBoolean());
+	}
+
+	private static void compareFailedCheckpoint(FailedCheckpointStats failed, JsonNode failedNode) {
+		assertEquals(failed.getCheckpointId(), failedNode.get("id").asLong());
+		assertEquals(failed.getStatus().toString(), failedNode.get("status").asText());
+		assertEquals(failed.getProperties().isSavepoint(), failedNode.get("is_savepoint").asBoolean());
+		assertEquals(failed.getTriggerTimestamp(), failedNode.get("trigger_timestamp").asLong());
+		assertEquals(failed.getLatestAckTimestamp(), failedNode.get("latest_ack_timestamp").asLong());
+		assertEquals(failed.getStateSize(), failedNode.get("state_size").asLong());
+		assertEquals(failed.getEndToEndDuration(), failedNode.get("end_to_end_duration").asLong());
+		assertEquals(failed.getAlignmentBuffered(), failedNode.get("alignment_buffered").asLong());
+		assertEquals(failed.getNumberOfSubtasks(), failedNode.get("num_subtasks").asInt());
+		assertEquals(failed.getNumberOfAcknowledgedSubtasks(), failedNode.get("num_acknowledged_subtasks").asInt());
+
+		assertEquals(failed.getFailureTimestamp(), failedNode.get("failure_timestamp").asLong());
+		assertEquals(failed.getFailureMessage(), failedNode.get("failure_message").asText());
+	}
+
+	private static class TestCheckpointStats {
+		public final AccessExecutionGraph graph;
+		public final CheckpointStatsCounts counts;
+		public final MinMaxAvgStats stateSizeSummary;
+		public final MinMaxAvgStats durationSummary;
+		public final MinMaxAvgStats alignmentBufferedSummary;
+		public final CompletedCheckpointStatsSummary summary;
+		public final CompletedCheckpointStats latestCompleted;
+		public final CompletedCheckpointStats latestSavepoint;
+		public final FailedCheckpointStats latestFailed;
+		public final RestoredCheckpointStats latestRestored;
+		public final PendingCheckpointStats inProgress;
+		public final CompletedCheckpointStats completedSavepoint;
+		public final FailedCheckpointStats failed;
+		public final CheckpointStatsHistory history;
+		public final CheckpointStatsSnapshot snapshot;
+
+		public TestCheckpointStats(
+				AccessExecutionGraph graph,
+				CheckpointStatsCounts counts,
+				MinMaxAvgStats stateSizeSummary,
+				MinMaxAvgStats durationSummary,
+				MinMaxAvgStats alignmentBufferedSummary,
+				CompletedCheckpointStatsSummary summary,
+				CompletedCheckpointStats latestCompleted,
+				CompletedCheckpointStats latestSavepoint,
+				FailedCheckpointStats latestFailed,
+				RestoredCheckpointStats latestRestored,
+				PendingCheckpointStats inProgress,
+				CompletedCheckpointStats completedSavepoint,
+				FailedCheckpointStats failed,
+				CheckpointStatsHistory history,
+				CheckpointStatsSnapshot snapshot) {
+			this.graph = graph;
+			this.counts = counts;
+			this.stateSizeSummary = stateSizeSummary;
+			this.durationSummary = durationSummary;
+			this.alignmentBufferedSummary = alignmentBufferedSummary;
+			this.summary = summary;
+			this.latestCompleted = latestCompleted;
+			this.latestSavepoint = latestSavepoint;
+			this.latestFailed = latestFailed;
+			this.latestRestored = latestRestored;
+			this.inProgress = inProgress;
+			this.completedSavepoint = completedSavepoint;
+			this.failed = failed;
+			this.history = history;
+			this.snapshot = snapshot;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
new file mode 100644
index 0000000..4d9b394
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
@@ -0,0 +1,389 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
+import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
+import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
+import org.apache.flink.runtime.checkpoint.SubtaskStateStats;
+import org.apache.flink.runtime.checkpoint.TaskStateStats;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the CheckpointStatsSubtaskDetailsHandler.
+ */
+public class CheckpointStatsSubtaskDetailsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new CheckpointStatsDetailsSubtasksHandler.CheckpointStatsDetailsSubtasksJsonArchivist();
+		ObjectMapper mapper = new ObjectMapper();
+
+		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
+		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
+
+		TaskStateStats task = createTaskStateStats(1237);
+		when(checkpoint.getAllTaskStateStats()).thenReturn(Collections.singletonList(task));
+
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpoints()).thenReturn(Collections.<AbstractCheckpointStats>singletonList(checkpoint));
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+		when(graph.getJobID()).thenReturn(new JobID());
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals(
+			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + checkpoint.getCheckpointId() + "/subtasks/" + task.getJobVertexId(),
+			archive.getPath());
+		JsonNode rootNode = mapper.readTree(archive.getJson());
+		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
+		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
+
+		verifyTaskNode(rootNode, task, checkpoint.getTriggerTimestamp());
+	}
+
+	@Test
+	public void testGetPaths() {
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", paths[0]);
+	}
+
+	/**
+	 * Tests a subtask details request.
+	 */
+	@Test
+	public void testSubtaskRequest() throws Exception {
+		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
+		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
+
+		TaskStateStats task = createTaskStateStats(1237);
+		when(checkpoint.getTaskStateStats(any(JobVertexID.class))).thenReturn(task);
+
+		JsonNode rootNode = triggerRequest(checkpoint);
+		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
+		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
+
+		verifyTaskNode(rootNode, task, checkpoint.getTriggerTimestamp());
+	}
+
+	/**
+	 * Tests a subtask details request.
+	 */
+	@Test
+	public void testSubtaskRequestNoSummary() throws Exception {
+		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
+		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
+		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
+		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
+
+		TaskStateStats task = createTaskStateStats(0); // no acknowledged
+		when(checkpoint.getTaskStateStats(any(JobVertexID.class))).thenReturn(task);
+
+		JsonNode rootNode = triggerRequest(checkpoint);
+		assertNull(rootNode.get("summary"));
+	}
+
+	/**
+	 * Tests request with illegal checkpoint ID param.
+	 */
+	@Test
+	public void testIllegalCheckpointId() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "illegal checkpoint");
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Tests request with missing checkpoint ID param.
+	 */
+	@Test
+	public void testNoCheckpointIdParam() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Test lookup of not existing checkpoint in history.
+	 */
+	@Test
+	public void testCheckpointNotFound() throws Exception {
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpointById(anyLong())).thenReturn(null); // not found
+
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "123");
+		params.put("vertexid", new JobVertexID().toString());
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+		verify(history, times(1)).getCheckpointById(anyLong());
+	}
+
+	/**
+	 * Tests request with illegal job vertex ID param.
+	 */
+	@Test
+	public void testIllegalJobVertexIdParam() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "1");
+		params.put("vertexid", "illegal vertex id");
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Tests request with missing job vertex ID param.
+	 */
+	@Test
+	public void testNoJobVertexIdParam() throws Exception {
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "1");
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+	}
+
+	/**
+	 * Test lookup of not existing job vertex ID in checkpoint.
+	 */
+	@Test
+	public void testJobVertexNotFound() throws Exception {
+		PendingCheckpointStats inProgress = mock(PendingCheckpointStats.class);
+		when(inProgress.getTaskStateStats(any(JobVertexID.class))).thenReturn(null); // not found
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpointById(anyLong())).thenReturn(inProgress);
+
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "123");
+		params.put("vertexid", new JobVertexID().toString());
+		String json = handler.handleRequest(graph, params).get();
+
+		assertEquals("{}", json);
+		verify(inProgress, times(1)).getTaskStateStats(any(JobVertexID.class));
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throws Exception {
+		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
+		when(history.getCheckpointById(anyLong())).thenReturn(checkpoint);
+		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
+		when(snapshot.getHistory()).thenReturn(history);
+
+		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
+		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
+
+		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
+		Map<String, String> params = new HashMap<>();
+		params.put("checkpointid", "123");
+		params.put("vertexid", new JobVertexID().toString());
+		String json = handler.handleRequest(graph, params).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		return mapper.readTree(json);
+	}
+
+	private static TaskStateStats createTaskStateStats(int numAcknowledged) {
+		ThreadLocalRandom rand = ThreadLocalRandom.current();
+
+		TaskStateStats task = mock(TaskStateStats.class);
+		when(task.getJobVertexId()).thenReturn(new JobVertexID());
+		when(task.getLatestAckTimestamp()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getStateSize()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getAlignmentBuffered()).thenReturn(rand.nextLong(1024) + 1);
+		when(task.getNumberOfSubtasks()).thenReturn(rand.nextInt(1024) + 1);
+		when(task.getNumberOfAcknowledgedSubtasks()).thenReturn(numAcknowledged);
+
+		TaskStateStats.TaskStateStatsSummary summary = mock(TaskStateStats.TaskStateStatsSummary.class);
+
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getStateSizeStats();
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getAckTimestampStats();
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getAlignmentBufferedStats();
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getAlignmentDurationStats();
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getSyncCheckpointDurationStats();
+		doReturn(createMinMaxAvgStats(rand)).when(summary).getAsyncCheckpointDurationStats();
+
+		when(task.getSummaryStats()).thenReturn(summary);
+
+		SubtaskStateStats[] subtasks = new SubtaskStateStats[3];
+		subtasks[0] = createSubtaskStats(0, rand);
+		subtasks[1] = createSubtaskStats(1, rand);
+		subtasks[2] = null;
+
+		when(task.getSubtaskStats()).thenReturn(subtasks);
+
+		return task;
+	}
+
+	private static void verifyTaskNode(JsonNode taskNode, TaskStateStats task, long triggerTimestamp) {
+		long duration = ThreadLocalRandom.current().nextInt(128);
+
+		assertEquals(task.getLatestAckTimestamp(), taskNode.get("latest_ack_timestamp").asLong());
+		assertEquals(task.getStateSize(), taskNode.get("state_size").asLong());
+		assertEquals(task.getEndToEndDuration(task.getLatestAckTimestamp() - duration), taskNode.get("end_to_end_duration").asLong());
+		assertEquals(task.getAlignmentBuffered(), taskNode.get("alignment_buffered").asLong());
+		assertEquals(task.getNumberOfSubtasks(), taskNode.get("num_subtasks").asInt());
+		assertEquals(task.getNumberOfAcknowledgedSubtasks(), taskNode.get("num_acknowledged_subtasks").asInt());
+
+		TaskStateStats.TaskStateStatsSummary summary = task.getSummaryStats();
+		verifyMinMaxAvgStats(summary.getStateSizeStats(), taskNode.get("summary").get("state_size"));
+		verifyMinMaxAvgStats(summary.getSyncCheckpointDurationStats(), taskNode.get("summary").get("checkpoint_duration").get("sync"));
+		verifyMinMaxAvgStats(summary.getAsyncCheckpointDurationStats(), taskNode.get("summary").get("checkpoint_duration").get("async"));
+		verifyMinMaxAvgStats(summary.getAlignmentBufferedStats(), taskNode.get("summary").get("alignment").get("buffered"));
+		verifyMinMaxAvgStats(summary.getAlignmentDurationStats(), taskNode.get("summary").get("alignment").get("duration"));
+
+		JsonNode endToEndDurationNode = taskNode.get("summary").get("end_to_end_duration");
+		assertEquals(summary.getAckTimestampStats().getMinimum() - triggerTimestamp, endToEndDurationNode.get("min").asLong());
+		assertEquals(summary.getAckTimestampStats().getMaximum() - triggerTimestamp, endToEndDurationNode.get("max").asLong());
+		assertEquals((long) summary.getAckTimestampStats().getAverage() - triggerTimestamp, endToEndDurationNode.get("avg").asLong());
+
+		SubtaskStateStats[] subtasks = task.getSubtaskStats();
+		Iterator<JsonNode> it = taskNode.get("subtasks").iterator();
+
+		assertTrue(it.hasNext());
+		verifySubtaskStats(it.next(), 0, subtasks[0]);
+
+		assertTrue(it.hasNext());
+		verifySubtaskStats(it.next(), 1, subtasks[1]);
+
+		assertTrue(it.hasNext());
+		verifySubtaskStats(it.next(), 2, subtasks[2]);
+
+		assertFalse(it.hasNext());
+	}
+
+	private static SubtaskStateStats createSubtaskStats(int index, ThreadLocalRandom rand) {
+		SubtaskStateStats subtask = mock(SubtaskStateStats.class);
+		when(subtask.getSubtaskIndex()).thenReturn(index);
+		when(subtask.getAckTimestamp()).thenReturn(rand.nextLong(1024));
+		when(subtask.getAlignmentBuffered()).thenReturn(rand.nextLong(1024));
+		when(subtask.getAlignmentDuration()).thenReturn(rand.nextLong(1024));
+		when(subtask.getSyncCheckpointDuration()).thenReturn(rand.nextLong(1024));
+		when(subtask.getAsyncCheckpointDuration()).thenReturn(rand.nextLong(1024));
+		when(subtask.getAckTimestamp()).thenReturn(rand.nextLong(1024));
+		when(subtask.getStateSize()).thenReturn(rand.nextLong(1024));
+		when(subtask.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024));
+		return subtask;
+	}
+
+	private static void verifySubtaskStats(JsonNode subtaskNode, int index, SubtaskStateStats subtask) {
+		if (subtask == null) {
+			assertEquals(index, subtaskNode.get("index").asInt());
+			assertEquals("pending_or_failed", subtaskNode.get("status").asText());
+		} else {
+			assertEquals(subtask.getSubtaskIndex(), subtaskNode.get("index").asInt());
+			assertEquals("completed", subtaskNode.get("status").asText());
+			assertEquals(subtask.getAckTimestamp(), subtaskNode.get("ack_timestamp").asLong());
+			assertEquals(subtask.getEndToEndDuration(0), subtaskNode.get("end_to_end_duration").asLong());
+			assertEquals(subtask.getStateSize(), subtaskNode.get("state_size").asLong());
+			assertEquals(subtask.getSyncCheckpointDuration(), subtaskNode.get("checkpoint").get("sync").asLong());
+			assertEquals(subtask.getAsyncCheckpointDuration(), subtaskNode.get("checkpoint").get("async").asLong());
+			assertEquals(subtask.getAlignmentBuffered(), subtaskNode.get("alignment").get("buffered").asLong());
+			assertEquals(subtask.getAlignmentDuration(), subtaskNode.get("alignment").get("duration").asLong());
+		}
+	}
+
+	private static MinMaxAvgStats createMinMaxAvgStats(ThreadLocalRandom rand) {
+		MinMaxAvgStats mma = mock(MinMaxAvgStats.class);
+		when(mma.getMinimum()).thenReturn(rand.nextLong(1024));
+		when(mma.getMaximum()).thenReturn(rand.nextLong(1024));
+		when(mma.getAverage()).thenReturn(rand.nextLong(1024));
+
+		return mma;
+	}
+
+	private static void verifyMinMaxAvgStats(MinMaxAvgStats expected, JsonNode node) {
+		assertEquals(expected.getMinimum(), node.get("min").asLong());
+		assertEquals(expected.getMaximum(), node.get("max").asLong());
+		assertEquals(expected.getAverage(), node.get("avg").asLong());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/MimeTypesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/MimeTypesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/MimeTypesTest.java
new file mode 100644
index 0000000..8e5ea17
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/MimeTypesTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.handler.legacy.files;
+
+import org.apache.flink.runtime.rest.handler.util.MimeTypes;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the MIME types map.
+ */
+public class MimeTypesTest {
+
+	@Test
+	public void testCompleteness() {
+		try {
+			assertNotNull(MimeTypes.getMimeTypeForExtension("txt"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("htm"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("html"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("css"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("js"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("json"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("png"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("jpg"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("jpeg"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("gif"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("woff"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("woff2"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("otf"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("ttf"));
+			assertNotNull(MimeTypes.getMimeTypeForExtension("eot"));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testFileNameExtraction() {
+		try {
+			assertNotNull(MimeTypes.getMimeTypeForFileName("test.txt"));
+			assertNotNull(MimeTypes.getMimeTypeForFileName("t.txt"));
+			assertNotNull(MimeTypes.getMimeTypeForFileName("first.second.third.txt"));
+
+			assertNull(MimeTypes.getMimeTypeForFileName(".txt"));
+			assertNull(MimeTypes.getMimeTypeForFileName("txt"));
+			assertNull(MimeTypes.getMimeTypeForFileName("test."));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandlerTest.java
new file mode 100644
index 0000000..7cbbdc1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandlerTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+/**
+ * Tests for the AbstractMetricsHandler.
+ */
+public class AbstractMetricsHandlerTest extends TestLogger {
+	/**
+	 * Verifies that the handlers correctly handle expected REST calls.
+	 */
+	@Test
+	public void testHandleRequest() throws Exception {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		Map<String, String> queryParams = new HashMap<>();
+
+		pathParams.put("jobid", "jobid");
+		pathParams.put("vertexid", "taskid");
+
+		// get list of available metrics
+		String availableList = handler.handleJsonRequest(pathParams, queryParams, null).get();
+
+		assertEquals("[" +
+				"{\"id\":\"8.opname.abc.metric5\"}," +
+				"{\"id\":\"8.abc.metric4\"}" +
+				"]",
+			availableList);
+
+		// get value for a single metric
+		queryParams.put("get", "8.opname.abc.metric5");
+
+		String metricValue = handler.handleJsonRequest(pathParams, queryParams, null).get();
+
+		assertEquals("[" +
+				"{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}" +
+				"]"
+			, metricValue
+		);
+
+		// get values for multiple metrics
+		queryParams.put("get", "8.opname.abc.metric5,8.abc.metric4");
+
+		String metricValues = handler.handleJsonRequest(pathParams, queryParams, null).get();
+
+		assertEquals("[" +
+				"{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}," +
+				"{\"id\":\"8.abc.metric4\",\"value\":\"3\"}" +
+				"]",
+			metricValues
+		);
+	}
+
+	/**
+	 * Verifies that a malformed request for available metrics does not throw an exception.
+	 */
+	@Test
+	public void testInvalidListDoesNotFail() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		Map<String, String> queryParams = new HashMap<>();
+
+		pathParams.put("jobid", "jobid");
+		pathParams.put("vertexid", "taskid");
+
+		//-----invalid variable
+		pathParams.put("jobid", "nonexistent");
+
+		try {
+			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
+		} catch (Exception e) {
+			fail();
+		}
+	}
+
+	/**
+	 * Verifies that a malformed request for a metric value does not throw an exception.
+	 */
+	@Test
+	public void testInvalidGetDoesNotFail() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		Map<String, String> queryParams = new HashMap<>();
+
+		pathParams.put("jobid", "jobid");
+		pathParams.put("vertexid", "taskid");
+
+		//-----empty string
+		queryParams.put("get", "");
+
+		try {
+			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
+		} catch (Exception e) {
+			fail(e.getMessage());
+		}
+
+		//-----invalid variable
+		pathParams.put("jobid", "nonexistent");
+		queryParams.put("get", "subindex.opname.abc.metric5");
+
+		try {
+			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
+		} catch (Exception e) {
+			fail(e.getMessage());
+		}
+
+		//-----invalid metric
+		pathParams.put("jobid", "nonexistant");
+		queryParams.put("get", "subindex.opname.abc.nonexistant");
+
+		try {
+			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
+		} catch (Exception e) {
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandlerTest.java
new file mode 100644
index 0000000..95ac271
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandlerTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+/**
+ * Tests for the JobManagerMetricsHandler.
+ */
+public class JobManagerMetricsHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobmanager/metrics", paths[0]);
+	}
+
+	@Test
+	public void getMapFor() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertEquals("0", metrics.get("abc.metric1"));
+	}
+
+	@Test
+	public void getMapForNull() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = fetcher.getMetricStore();
+
+		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertNotNull(metrics);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandlerTest.java
new file mode 100644
index 0000000..4b28e65
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandlerTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.runtime.rest.handler.legacy.metrics.JobMetricsHandler.PARAMETER_JOB_ID;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+/**
+ * Tests for the JobMetricsHandler.
+ */
+public class JobMetricsHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/metrics", paths[0]);
+	}
+
+	@Test
+	public void getMapFor() throws Exception {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		pathParams.put(PARAMETER_JOB_ID, "jobid");
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertEquals("2", metrics.get("abc.metric3"));
+	}
+
+	@Test
+	public void getMapForNull() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = fetcher.getMetricStore();
+
+		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertNull(metrics);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandlerTest.java
new file mode 100644
index 0000000..c1304c4
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandlerTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.runtime.rest.handler.legacy.metrics.JobMetricsHandler.PARAMETER_JOB_ID;
+import static org.apache.flink.runtime.rest.handler.legacy.metrics.JobVertexMetricsHandler.PARAMETER_VERTEX_ID;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+/**
+ * Tests for the JobVertexMetricsHandler.
+ */
+public class JobVertexMetricsHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/metrics", paths[0]);
+	}
+
+	@Test
+	public void getMapFor() throws Exception {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		pathParams.put(PARAMETER_JOB_ID, "jobid");
+		pathParams.put(PARAMETER_VERTEX_ID, "taskid");
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertEquals("3", metrics.get("8.abc.metric4"));
+
+		assertEquals("4", metrics.get("8.opname.abc.metric5"));
+	}
+
+	@Test
+	public void getMapForNull() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = fetcher.getMetricStore();
+
+		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertNull(metrics);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java
new file mode 100644
index 0000000..b278979
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
+import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
+import org.apache.flink.runtime.metrics.dump.MetricQueryService;
+import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
+import org.apache.flink.runtime.metrics.util.TestingHistogram;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaJobManagerRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.eq;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/**
+ * Tests for the MetricFetcher.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(MetricFetcher.class)
+public class MetricFetcherTest extends TestLogger {
+	@Test
+	public void testUpdate() throws Exception {
+		final Time timeout = Time.seconds(10L);
+
+		// ========= setup TaskManager =================================================================================
+		JobID jobID = new JobID();
+		InstanceID tmID = new InstanceID();
+		ResourceID tmRID = new ResourceID(tmID.toString());
+		TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class);
+		when(taskManagerGateway.getAddress()).thenReturn("/tm/address");
+
+		Instance taskManager = mock(Instance.class);
+		when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway);
+		when(taskManager.getId()).thenReturn(tmID);
+		when(taskManager.getTaskManagerID()).thenReturn(tmRID);
+
+		// ========= setup JobManager ==================================================================================
+		JobDetails details = mock(JobDetails.class);
+		when(details.getJobId()).thenReturn(jobID);
+
+		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
+
+		when(jobManagerGateway.requestJobDetails(anyBoolean(), anyBoolean(), any(Time.class)))
+			.thenReturn(CompletableFuture.completedFuture(new MultipleJobsDetails(new JobDetails[0], new JobDetails[0])));
+		when(jobManagerGateway.requestTaskManagerInstances(any(Time.class)))
+			.thenReturn(CompletableFuture.completedFuture(Collections.singleton(taskManager)));
+		when(jobManagerGateway.getAddress()).thenReturn("/jm/address");
+
+		GatewayRetriever<JobManagerGateway> retriever = mock(AkkaJobManagerRetriever.class);
+		when(retriever.getNow())
+			.thenReturn(Optional.of(jobManagerGateway));
+
+		// ========= setup QueryServices ================================================================================
+		MetricQueryServiceGateway jmQueryService = mock(MetricQueryServiceGateway.class);
+		MetricQueryServiceGateway tmQueryService = mock(MetricQueryServiceGateway.class);
+
+		MetricDumpSerialization.MetricSerializationResult requestMetricsAnswer = createRequestDumpAnswer(tmID, jobID);
+
+		when(jmQueryService.queryMetrics(any(Time.class)))
+			.thenReturn(CompletableFuture.completedFuture(new MetricDumpSerialization.MetricSerializationResult(new byte[0], 0, 0, 0, 0)));
+		when(tmQueryService.queryMetrics(any(Time.class)))
+			.thenReturn(CompletableFuture.completedFuture(requestMetricsAnswer));
+
+		MetricQueryServiceRetriever queryServiceRetriever = mock(MetricQueryServiceRetriever.class);
+		when(queryServiceRetriever.retrieveService(eq("/jm/" + MetricQueryService.METRIC_QUERY_SERVICE_NAME))).thenReturn(CompletableFuture.completedFuture(jmQueryService));
+		when(queryServiceRetriever.retrieveService(eq("/tm/" + MetricQueryService.METRIC_QUERY_SERVICE_NAME + "_" + tmRID.getResourceIdString()))).thenReturn(CompletableFuture.completedFuture(tmQueryService));
+
+		// ========= start MetricFetcher testing =======================================================================
+		MetricFetcher fetcher = new MetricFetcher(
+			retriever,
+			queryServiceRetriever,
+			Executors.directExecutor(),
+			timeout);
+
+		// verify that update fetches metrics and updates the store
+		fetcher.update();
+		MetricStore store = fetcher.getMetricStore();
+		synchronized (store) {
+			assertEquals("7", store.jobManager.metrics.get("abc.hist_min"));
+			assertEquals("6", store.jobManager.metrics.get("abc.hist_max"));
+			assertEquals("4.0", store.jobManager.metrics.get("abc.hist_mean"));
+			assertEquals("0.5", store.jobManager.metrics.get("abc.hist_median"));
+			assertEquals("5.0", store.jobManager.metrics.get("abc.hist_stddev"));
+			assertEquals("0.75", store.jobManager.metrics.get("abc.hist_p75"));
+			assertEquals("0.9", store.jobManager.metrics.get("abc.hist_p90"));
+			assertEquals("0.95", store.jobManager.metrics.get("abc.hist_p95"));
+			assertEquals("0.98", store.jobManager.metrics.get("abc.hist_p98"));
+			assertEquals("0.99", store.jobManager.metrics.get("abc.hist_p99"));
+			assertEquals("0.999", store.jobManager.metrics.get("abc.hist_p999"));
+
+			assertEquals("x", store.getTaskManagerMetricStore(tmID.toString()).metrics.get("abc.gauge"));
+			assertEquals("5.0", store.getJobMetricStore(jobID.toString()).metrics.get("abc.jc"));
+			assertEquals("2", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.abc.tc"));
+			assertEquals("1", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.opname.abc.oc"));
+		}
+	}
+
+	private static MetricDumpSerialization.MetricSerializationResult createRequestDumpAnswer(InstanceID tmID, JobID jobID) {
+		Map<Counter, Tuple2<QueryScopeInfo, String>> counters = new HashMap<>();
+		Map<Gauge<?>, Tuple2<QueryScopeInfo, String>> gauges = new HashMap<>();
+		Map<Histogram, Tuple2<QueryScopeInfo, String>> histograms = new HashMap<>();
+		Map<Meter, Tuple2<QueryScopeInfo, String>> meters = new HashMap<>();
+
+		SimpleCounter c1 = new SimpleCounter();
+		SimpleCounter c2 = new SimpleCounter();
+
+		c1.inc(1);
+		c2.inc(2);
+
+		counters.put(c1, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.OperatorQueryScopeInfo(jobID.toString(), "taskid", 2, "opname", "abc"), "oc"));
+		counters.put(c2, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.TaskQueryScopeInfo(jobID.toString(), "taskid", 2, "abc"), "tc"));
+		meters.put(new Meter() {
+			@Override
+			public void markEvent() {
+			}
+
+			@Override
+			public void markEvent(long n) {
+			}
+
+			@Override
+			public double getRate() {
+				return 5;
+			}
+
+			@Override
+			public long getCount() {
+				return 10;
+			}
+		}, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.JobQueryScopeInfo(jobID.toString(), "abc"), "jc"));
+		gauges.put(new Gauge<String>() {
+			@Override
+			public String getValue() {
+				return "x";
+			}
+		}, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmID.toString(), "abc"), "gauge"));
+		histograms.put(new TestingHistogram(), new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.JobManagerQueryScopeInfo("abc"), "hist"));
+
+		MetricDumpSerialization.MetricDumpSerializer serializer = new MetricDumpSerialization.MetricDumpSerializer();
+		MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, gauges, histograms, meters);
+		serializer.close();
+
+		return dump;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStoreTest.java
new file mode 100644
index 0000000..2e83e08
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStoreTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.metrics.dump.MetricDump;
+import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the MetricStore.
+ */
+public class MetricStoreTest extends TestLogger {
+	@Test
+	public void testAdd() throws IOException {
+		MetricStore store = setupStore(new MetricStore());
+
+		assertEquals("0", store.getJobManagerMetricStore().getMetric("abc.metric1", "-1"));
+		assertEquals("1", store.getTaskManagerMetricStore("tmid").getMetric("abc.metric2", "-1"));
+		assertEquals("2", store.getJobMetricStore("jobid").getMetric("abc.metric3", "-1"));
+		assertEquals("3", store.getTaskMetricStore("jobid", "taskid").getMetric("8.abc.metric4", "-1"));
+		assertEquals("4", store.getTaskMetricStore("jobid", "taskid").getMetric("8.opname.abc.metric5", "-1"));
+	}
+
+	@Test
+	public void testMalformedNameHandling() {
+		MetricStore store = new MetricStore();
+		//-----verify that no exceptions are thrown
+
+		// null
+		store.add(null);
+		// empty name
+		QueryScopeInfo.JobManagerQueryScopeInfo info = new QueryScopeInfo.JobManagerQueryScopeInfo("");
+		MetricDump.CounterDump cd = new MetricDump.CounterDump(info, "", 0);
+		store.add(cd);
+
+		//-----verify that no side effects occur
+		assertEquals(0, store.jobManager.metrics.size());
+		assertEquals(0, store.taskManagers.size());
+		assertEquals(0, store.jobs.size());
+	}
+
+	public static MetricStore setupStore(MetricStore store) {
+		QueryScopeInfo.JobManagerQueryScopeInfo jm = new QueryScopeInfo.JobManagerQueryScopeInfo("abc");
+		MetricDump.CounterDump cd1 = new MetricDump.CounterDump(jm, "metric1", 0);
+
+		QueryScopeInfo.TaskManagerQueryScopeInfo tm = new QueryScopeInfo.TaskManagerQueryScopeInfo("tmid", "abc");
+		MetricDump.CounterDump cd2 = new MetricDump.CounterDump(tm, "metric2", 1);
+
+		QueryScopeInfo.JobQueryScopeInfo job = new QueryScopeInfo.JobQueryScopeInfo("jobid", "abc");
+		MetricDump.CounterDump cd3 = new MetricDump.CounterDump(job, "metric3", 2);
+
+		QueryScopeInfo.TaskQueryScopeInfo task = new QueryScopeInfo.TaskQueryScopeInfo("jobid", "taskid", 8, "abc");
+		MetricDump.CounterDump cd4 = new MetricDump.CounterDump(task, "metric4", 3);
+
+		QueryScopeInfo.OperatorQueryScopeInfo operator = new QueryScopeInfo.OperatorQueryScopeInfo("jobid", "taskid", 8, "opname", "abc");
+		MetricDump.CounterDump cd5 = new MetricDump.CounterDump(operator, "metric5", 4);
+
+		store.add(cd1);
+		store.add(cd2);
+		store.add(cd3);
+		store.add(cd4);
+		store.add(cd5);
+
+		return store;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandlerTest.java
new file mode 100644
index 0000000..c6e8f07
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandlerTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.runtime.rest.handler.legacy.TaskManagersHandler.TASK_MANAGER_ID_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+/**
+ * Tests for the TaskManagerMetricsHandler.
+ */
+public class TaskManagerMetricsHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/taskmanagers/:taskmanagerid/metrics", paths[0]);
+	}
+
+	@Test
+	public void getMapFor() throws Exception {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
+
+		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+		pathParams.put(TASK_MANAGER_ID_KEY, "tmid");
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertEquals("1", metrics.get("abc.metric2"));
+	}
+
+	@Test
+	public void getMapForNull() {
+		MetricFetcher fetcher = new MetricFetcher(
+			mock(GatewayRetriever.class),
+			mock(MetricQueryServiceRetriever.class),
+			Executors.directExecutor(),
+			TestingUtils.TIMEOUT());
+		MetricStore store = fetcher.getMetricStore();
+
+		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher);
+
+		Map<String, String> pathParams = new HashMap<>();
+
+		Map<String, String> metrics = handler.getMapFor(pathParams, store);
+
+		assertNull(metrics);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionBuilder.java
new file mode 100644
index 0000000..ad5cd6b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionBuilder.java
@@ -0,0 +1,150 @@
+/*
+ * 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.rest.handler.legacy.utils;
+
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.ArchivedExecution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.Preconditions;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+/**
+ * Utility class for constructing an ArchivedExecution.
+ */
+public class ArchivedExecutionBuilder {
+	private ExecutionAttemptID attemptId;
+	private long[] stateTimestamps;
+	private int attemptNumber;
+	private ExecutionState state;
+	private String failureCause;
+	private TaskManagerLocation assignedResourceLocation;
+	private StringifiedAccumulatorResult[] userAccumulators;
+	private IOMetrics ioMetrics;
+	private int parallelSubtaskIndex;
+
+	public ArchivedExecutionBuilder setAttemptId(ExecutionAttemptID attemptId) {
+		this.attemptId = attemptId;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setStateTimestamps(long[] stateTimestamps) {
+		Preconditions.checkArgument(stateTimestamps.length == ExecutionState.values().length);
+		this.stateTimestamps = stateTimestamps;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setAttemptNumber(int attemptNumber) {
+		this.attemptNumber = attemptNumber;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setState(ExecutionState state) {
+		this.state = state;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setFailureCause(String failureCause) {
+		this.failureCause = failureCause;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setAssignedResourceLocation(TaskManagerLocation assignedResourceLocation) {
+		this.assignedResourceLocation = assignedResourceLocation;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setUserAccumulators(StringifiedAccumulatorResult[] userAccumulators) {
+		this.userAccumulators = userAccumulators;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setParallelSubtaskIndex(int parallelSubtaskIndex) {
+		this.parallelSubtaskIndex = parallelSubtaskIndex;
+		return this;
+	}
+
+	public ArchivedExecutionBuilder setIOMetrics(IOMetrics ioMetrics) {
+		this.ioMetrics = ioMetrics;
+		return this;
+	}
+
+	public ArchivedExecution build() throws UnknownHostException {
+		return new ArchivedExecution(
+			userAccumulators != null ? userAccumulators : new StringifiedAccumulatorResult[0],
+			ioMetrics != null ? ioMetrics : new TestIOMetrics(),
+			attemptId != null ? attemptId : new ExecutionAttemptID(),
+			attemptNumber,
+			state != null ? state : ExecutionState.FINISHED,
+			failureCause != null ? failureCause : "(null)",
+			assignedResourceLocation != null ? assignedResourceLocation : new TaskManagerLocation(new ResourceID("tm"), InetAddress.getLocalHost(), 1234),
+			parallelSubtaskIndex,
+			stateTimestamps != null ? stateTimestamps : new long[]{1, 2, 3, 4, 5, 5, 5, 5}
+		);
+	}
+
+	private static class TestIOMetrics extends IOMetrics {
+		private static final long serialVersionUID = -5920076211680012555L;
+
+		public TestIOMetrics() {
+			super(
+				new MeterView(new TestCounter(1), 0),
+				new MeterView(new TestCounter(2), 0),
+				new MeterView(new TestCounter(3), 0),
+				new MeterView(new TestCounter(4), 0),
+				new MeterView(new TestCounter(5), 0));
+		}
+	}
+
+	private static class TestCounter implements Counter {
+		private final long count;
+
+		private TestCounter(long count) {
+			this.count = count;
+		}
+
+		@Override
+		public void inc() {
+		}
+
+		@Override
+		public void inc(long n) {
+		}
+
+		@Override
+		public void dec() {
+		}
+
+		@Override
+		public void dec(long n) {
+		}
+
+		@Override
+		public long getCount() {
+			return count;
+		}
+	}
+}


[06/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java
new file mode 100644
index 0000000..95d417a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java
@@ -0,0 +1,205 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricStore;
+import org.apache.flink.util.StringUtils;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A request handler that provides an overview over all taskmanagers or details for a single one.
+ */
+public class TaskManagersHandler extends AbstractJsonRequestHandler  {
+
+	private static final String TASKMANAGERS_REST_PATH = "/taskmanagers";
+	private static final String TASKMANAGER_DETAILS_REST_PATH = "/taskmanagers/:taskmanagerid";
+
+	public static final String TASK_MANAGER_ID_KEY = "taskmanagerid";
+
+	private final Time timeout;
+
+	private final MetricFetcher fetcher;
+
+	public TaskManagersHandler(Executor executor, Time timeout, MetricFetcher fetcher) {
+		super(executor);
+		this.timeout = requireNonNull(timeout);
+		this.fetcher = fetcher;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		if (jobManagerGateway != null) {
+			// whether one task manager's metrics are requested, or all task manager, we
+			// return them in an array. This avoids unnecessary code complexity.
+			// If only one task manager is requested, we only fetch one task manager metrics.
+			if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) {
+				InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(pathParams.get(TASK_MANAGER_ID_KEY)));
+				CompletableFuture<Optional<Instance>> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout);
+
+				return tmInstanceFuture.thenApplyAsync(
+					(Optional<Instance> optTaskManager) -> {
+						try {
+							return writeTaskManagersJson(
+								optTaskManager.map(Collections::singleton).orElse(Collections.emptySet()),
+								pathParams);
+						} catch (IOException e) {
+							throw new FlinkFutureException("Could not write TaskManagers JSON.", e);
+						}
+					},
+					executor);
+			} else {
+				CompletableFuture<Collection<Instance>> tmInstancesFuture = jobManagerGateway.requestTaskManagerInstances(timeout);
+
+				return tmInstancesFuture.thenApplyAsync(
+					(Collection<Instance> taskManagers) -> {
+						try {
+							return writeTaskManagersJson(taskManagers, pathParams);
+						} catch (IOException e) {
+							throw new FlinkFutureException("Could not write TaskManagers JSON.", e);
+						}
+					},
+					executor);
+			}
+		}
+		else {
+			return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
+		}
+	}
+
+	private String writeTaskManagersJson(Collection<Instance> instances, Map<String, String> pathParams) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+		gen.writeArrayFieldStart("taskmanagers");
+
+		for (Instance instance : instances) {
+			gen.writeStartObject();
+			gen.writeStringField("id", instance.getId().toString());
+			gen.writeStringField("path", instance.getTaskManagerGateway().getAddress());
+			gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort());
+			gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat());
+			gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots());
+			gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots());
+			gen.writeNumberField("cpuCores", instance.getResources().getNumberOfCPUCores());
+			gen.writeNumberField("physicalMemory", instance.getResources().getSizeOfPhysicalMemory());
+			gen.writeNumberField("freeMemory", instance.getResources().getSizeOfJvmHeap());
+			gen.writeNumberField("managedMemory", instance.getResources().getSizeOfManagedMemory());
+
+			// only send metrics when only one task manager requests them.
+			if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) {
+				fetcher.update();
+				MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString());
+				if (metrics != null) {
+					gen.writeObjectFieldStart("metrics");
+					long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
+					long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
+					long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));
+
+					gen.writeNumberField("heapCommitted", heapCommitted);
+					gen.writeNumberField("heapUsed", heapUsed);
+					gen.writeNumberField("heapMax", heapTotal);
+
+					long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
+					long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
+					long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));
+
+					gen.writeNumberField("nonHeapCommitted", nonHeapCommitted);
+					gen.writeNumberField("nonHeapUsed", nonHeapUsed);
+					gen.writeNumberField("nonHeapMax", nonHeapTotal);
+
+					gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted);
+					gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed);
+					gen.writeNumberField("totalMax", heapTotal + nonHeapTotal);
+
+					long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0"));
+					long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0"));
+					long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0"));
+
+					gen.writeNumberField("directCount", directCount);
+					gen.writeNumberField("directUsed", directUsed);
+					gen.writeNumberField("directMax", directMax);
+
+					long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0"));
+					long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0"));
+					long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0"));
+
+					gen.writeNumberField("mappedCount", mappedCount);
+					gen.writeNumberField("mappedUsed", mappedUsed);
+					gen.writeNumberField("mappedMax", mappedMax);
+
+					long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0"));
+					long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0"));
+
+					gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable);
+					gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal);
+
+					gen.writeArrayFieldStart("garbageCollectors");
+
+					for (String gcName : metrics.garbageCollectorNames) {
+						String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null);
+						String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null);
+						if (count != null  && time != null) {
+							gen.writeStartObject();
+							gen.writeStringField("name", gcName);
+							gen.writeNumberField("count", Long.valueOf(count));
+							gen.writeNumberField("time", Long.valueOf(time));
+							gen.writeEndObject();
+						}
+					}
+
+					gen.writeEndArray();
+					gen.writeEndObject();
+				}
+			}
+
+			gen.writeEndObject();
+		}
+
+		gen.writeEndArray();
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java
new file mode 100644
index 0000000..96bf7ec
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java
@@ -0,0 +1,333 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Back pressure statistics tracker.
+ *
+ * <p>Back pressure is determined by sampling running tasks. If a task is
+ * slowed down by back pressure it will be stuck in memory requests to a
+ * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}.
+ *
+ * <p>The back pressured stack traces look like this:
+ *
+ * <pre>
+ * java.lang.Object.wait(Native Method)
+ * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
+ * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) <--- BLOCKING
+ * request
+ * [...]
+ * </pre>
+ */
+public class BackPressureStatsTracker {
+
+	private static final Logger LOG = LoggerFactory.getLogger(BackPressureStatsTracker.class);
+
+	/** Maximum stack trace depth for samples. */
+	static final int MAX_STACK_TRACE_DEPTH = 3;
+
+	/** Expected class name for back pressure indicating stack trace element. */
+	static final String EXPECTED_CLASS_NAME = "org.apache.flink.runtime.io.network.buffer.LocalBufferPool";
+
+	/** Expected method name for back pressure indicating stack trace element. */
+	static final String EXPECTED_METHOD_NAME = "requestBufferBlocking";
+
+	/** Lock guarding trigger operations. */
+	private final Object lock = new Object();
+
+	/* Stack trace sample coordinator. */
+	private final StackTraceSampleCoordinator coordinator;
+
+	/**
+	 * Completed stats. Important: Job vertex IDs need to be scoped by job ID,
+	 * because they are potentially constant across runs messing up the cached
+	 * data.
+	 */
+	private final Cache<ExecutionJobVertex, OperatorBackPressureStats> operatorStatsCache;
+
+	/** Pending in progress stats. Important: Job vertex IDs need to be scoped
+	 * by job ID, because they are potentially constant across runs messing up
+	 * the cached data.*/
+	private final Set<ExecutionJobVertex> pendingStats = new HashSet<>();
+
+	/** Cleanup interval for completed stats cache. */
+	private final int cleanUpInterval;
+
+	private final int numSamples;
+
+	private final Time delayBetweenSamples;
+
+	/** Flag indicating whether the stats tracker has been shut down. */
+	private boolean shutDown;
+
+	/**
+	 * Creates a back pressure statistics tracker.
+	 *
+	 * @param cleanUpInterval     Clean up interval for completed stats.
+	 * @param numSamples          Number of stack trace samples when determining back pressure.
+	 * @param delayBetweenSamples Delay between samples when determining back pressure.
+	 */
+	public BackPressureStatsTracker(
+			StackTraceSampleCoordinator coordinator,
+			int cleanUpInterval,
+			int numSamples,
+			Time delayBetweenSamples) {
+
+		this.coordinator = checkNotNull(coordinator, "Stack trace sample coordinator");
+
+		checkArgument(cleanUpInterval >= 0, "Clean up interval");
+		this.cleanUpInterval = cleanUpInterval;
+
+		checkArgument(numSamples >= 1, "Number of samples");
+		this.numSamples = numSamples;
+
+		this.delayBetweenSamples = checkNotNull(delayBetweenSamples, "Delay between samples");
+
+		this.operatorStatsCache = CacheBuilder.newBuilder()
+				.concurrencyLevel(1)
+				.expireAfterAccess(cleanUpInterval, TimeUnit.MILLISECONDS)
+				.build();
+	}
+
+	/** Cleanup interval for completed stats cache. */
+	public long getCleanUpInterval() {
+		return cleanUpInterval;
+	}
+
+	/**
+	 * Returns back pressure statistics for a operator.
+	 *
+	 * @param vertex Operator to get the stats for.
+	 *
+	 * @return Back pressure statistics for an operator
+	 */
+	public Optional<OperatorBackPressureStats> getOperatorBackPressureStats(ExecutionJobVertex vertex) {
+		return Optional.ofNullable(operatorStatsCache.getIfPresent(vertex));
+	}
+
+	/**
+	 * Triggers a stack trace sample for a operator to gather the back pressure
+	 * statistics. If there is a sample in progress for the operator, the call
+	 * is ignored.
+	 *
+	 * @param vertex Operator to get the stats for.
+	 * @return Flag indicating whether a sample with triggered.
+	 */
+	@SuppressWarnings("unchecked")
+	public boolean triggerStackTraceSample(ExecutionJobVertex vertex) {
+		synchronized (lock) {
+			if (shutDown) {
+				return false;
+			}
+
+			if (!pendingStats.contains(vertex) &&
+					!vertex.getGraph().getState().isGloballyTerminalState()) {
+
+				Executor executor = vertex.getGraph().getFutureExecutor();
+
+				// Only trigger if still active job
+				if (executor != null) {
+					pendingStats.add(vertex);
+
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Triggering stack trace sample for tasks: " + Arrays.toString(vertex.getTaskVertices()));
+					}
+
+					CompletableFuture<StackTraceSample> sample = coordinator.triggerStackTraceSample(
+							vertex.getTaskVertices(),
+							numSamples,
+							delayBetweenSamples,
+							MAX_STACK_TRACE_DEPTH);
+
+					sample.handleAsync(new StackTraceSampleCompletionCallback(vertex), executor);
+
+					return true;
+				}
+			}
+
+			return false;
+		}
+	}
+
+	/**
+	 * Cleans up the operator stats cache if it contains timed out entries.
+	 *
+	 * <p>The Guava cache only evicts as maintenance during normal operations.
+	 * If this handler is inactive, it will never be cleaned.
+	 */
+	public void cleanUpOperatorStatsCache() {
+		operatorStatsCache.cleanUp();
+	}
+
+	/**
+	 * Shuts down the stats tracker.
+	 *
+	 * <p>Invalidates the cache and clears all pending stats.
+	 */
+	public void shutDown() {
+		synchronized (lock) {
+			if (!shutDown) {
+				operatorStatsCache.invalidateAll();
+				pendingStats.clear();
+
+				shutDown = true;
+			}
+		}
+	}
+
+	/**
+	 * Invalidates the cache (irrespective of clean up interval).
+	 */
+	void invalidateOperatorStatsCache() {
+		operatorStatsCache.invalidateAll();
+	}
+
+	/**
+	 * Callback on completed stack trace sample.
+	 */
+	class StackTraceSampleCompletionCallback implements BiFunction<StackTraceSample, Throwable, Void> {
+
+		private final ExecutionJobVertex vertex;
+
+		public StackTraceSampleCompletionCallback(ExecutionJobVertex vertex) {
+			this.vertex = vertex;
+		}
+
+		@Override
+		public Void apply(StackTraceSample stackTraceSample, Throwable throwable) {
+			synchronized (lock) {
+				try {
+					if (shutDown) {
+						return null;
+					}
+
+					// Job finished, ignore.
+					JobStatus jobState = vertex.getGraph().getState();
+					if (jobState.isGloballyTerminalState()) {
+						LOG.debug("Ignoring sample, because job is in state " + jobState + ".");
+					} else if (stackTraceSample != null) {
+						OperatorBackPressureStats stats = createStatsFromSample(stackTraceSample);
+						operatorStatsCache.put(vertex, stats);
+					} else {
+						LOG.debug("Failed to gather stack trace sample.", throwable);
+					}
+				} catch (Throwable t) {
+					LOG.error("Error during stats completion.", t);
+				} finally {
+					pendingStats.remove(vertex);
+				}
+
+				return null;
+			}
+		}
+
+		/**
+		 * Creates the back pressure stats from a stack trace sample.
+		 *
+		 * @param sample Stack trace sample to base stats on.
+		 *
+		 * @return Back pressure stats
+		 */
+		private OperatorBackPressureStats createStatsFromSample(StackTraceSample sample) {
+			Map<ExecutionAttemptID, List<StackTraceElement[]>> traces = sample.getStackTraces();
+
+			// Map task ID to subtask index, because the web interface expects
+			// it like that.
+			Map<ExecutionAttemptID, Integer> subtaskIndexMap = Maps
+					.newHashMapWithExpectedSize(traces.size());
+
+			Set<ExecutionAttemptID> sampledTasks = sample.getStackTraces().keySet();
+
+			for (ExecutionVertex task : vertex.getTaskVertices()) {
+				ExecutionAttemptID taskId = task.getCurrentExecutionAttempt().getAttemptId();
+				if (sampledTasks.contains(taskId)) {
+					subtaskIndexMap.put(taskId, task.getParallelSubtaskIndex());
+				} else {
+					LOG.debug("Outdated sample. A task, which is part of the " +
+							"sample has been reset.");
+				}
+			}
+
+			// Ratio of blocked samples to total samples per sub task. Array
+			// position corresponds to sub task index.
+			double[] backPressureRatio = new double[traces.size()];
+
+			for (Entry<ExecutionAttemptID, List<StackTraceElement[]>> entry : traces.entrySet()) {
+				int backPressureSamples = 0;
+
+				List<StackTraceElement[]> taskTraces = entry.getValue();
+
+				for (StackTraceElement[] trace : taskTraces) {
+					for (int i = trace.length - 1; i >= 0; i--) {
+						StackTraceElement elem = trace[i];
+
+						if (elem.getClassName().equals(EXPECTED_CLASS_NAME) &&
+								elem.getMethodName().equals(EXPECTED_METHOD_NAME)) {
+
+							backPressureSamples++;
+							break; // Continue with next stack trace
+						}
+					}
+				}
+
+				int subtaskIndex = subtaskIndexMap.get(entry.getKey());
+
+				int size = taskTraces.size();
+				double ratio = (size > 0)
+						? ((double) backPressureSamples) / size
+						: 0;
+
+				backPressureRatio[subtaskIndex] = ratio;
+			}
+
+			return new OperatorBackPressureStats(
+					sample.getSampleId(),
+					sample.getEndTime(),
+					backPressureRatio);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java
new file mode 100644
index 0000000..1a78a17
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java
@@ -0,0 +1,126 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Back pressure statistics of multiple tasks.
+ *
+ * <p>Statistics are gathered by sampling stack traces of running tasks. The
+ * back pressure ratio denotes the ratio of traces indicating back pressure
+ * to the total number of sampled traces.
+ */
+public class OperatorBackPressureStats {
+
+	/** ID of the corresponding sample. */
+	private final int sampleId;
+
+	/** End time stamp of the corresponding sample. */
+	private final long endTimestamp;
+
+	/** Back pressure ratio per subtask. */
+	private final double[] subTaskBackPressureRatio;
+
+	/** Maximum back pressure ratio. */
+	private final double maxSubTaskBackPressureRatio;
+
+	public OperatorBackPressureStats(
+			int sampleId,
+			long endTimestamp,
+			double[] subTaskBackPressureRatio) {
+
+		this.sampleId = sampleId;
+		this.endTimestamp = endTimestamp;
+		this.subTaskBackPressureRatio = checkNotNull(subTaskBackPressureRatio, "Sub task back pressure ratio");
+		checkArgument(subTaskBackPressureRatio.length >= 1, "No Sub task back pressure ratio specified");
+
+		double max = 0;
+		for (double ratio : subTaskBackPressureRatio) {
+			if (ratio > max) {
+				max = ratio;
+			}
+		}
+
+		maxSubTaskBackPressureRatio = max;
+	}
+
+	/**
+	 * Returns the ID of the sample.
+	 *
+	 * @return ID of the sample
+	 */
+	public int getSampleId() {
+		return sampleId;
+	}
+
+	/**
+	 * Returns the time stamp, when all stack traces were collected at the
+	 * JobManager.
+	 *
+	 * @return Time stamp, when all stack traces were collected at the
+	 * JobManager
+	 */
+	public long getEndTimestamp() {
+		return endTimestamp;
+	}
+
+	/**
+	 * Returns the number of sub tasks.
+	 *
+	 * @return Number of sub tasks.
+	 */
+	public int getNumberOfSubTasks() {
+		return subTaskBackPressureRatio.length;
+	}
+
+	/**
+	 * Returns the ratio of stack traces indicating back pressure to total
+	 * number of sampled stack traces.
+	 *
+	 * @param index Subtask index.
+	 *
+	 * @return Ratio of stack traces indicating back pressure to total number
+	 * of sampled stack traces.
+	 */
+	public double getBackPressureRatio(int index) {
+		return subTaskBackPressureRatio[index];
+	}
+
+	/**
+	 * Returns the maximum back pressure ratio of all sub tasks.
+	 *
+	 * @return Maximum back pressure ratio of all sub tasks.
+	 */
+	public double getMaxBackPressureRatio() {
+		return maxSubTaskBackPressureRatio;
+	}
+
+	@Override
+	public String toString() {
+		return "OperatorBackPressureStats{" +
+				"sampleId=" + sampleId +
+				", endTimestamp=" + endTimestamp +
+				", subTaskBackPressureRatio=" + Arrays.toString(subTaskBackPressureRatio) +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java
new file mode 100644
index 0000000..dda4e33
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java
@@ -0,0 +1,119 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * A sample of stack traces for one or more tasks.
+ *
+ * <p>The sampling is triggered in {@link StackTraceSampleCoordinator}.
+ */
+public class StackTraceSample {
+
+	/** ID of this sample (unique per job). */
+	private final int sampleId;
+
+	/** Time stamp, when the sample was triggered. */
+	private final long startTime;
+
+	/** Time stamp, when all stack traces were collected at the JobManager. */
+	private final long endTime;
+
+	/** Map of stack traces by execution ID. */
+	private final Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask;
+
+	/**
+	 * Creates a stack trace sample.
+	 *
+	 * @param sampleId          ID of the sample.
+	 * @param startTime         Time stamp, when the sample was triggered.
+	 * @param endTime           Time stamp, when all stack traces were
+	 *                          collected at the JobManager.
+	 * @param stackTracesByTask Map of stack traces by execution ID.
+	 */
+	public StackTraceSample(
+			int sampleId,
+			long startTime,
+			long endTime,
+			Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask) {
+
+		checkArgument(sampleId >= 0, "Negative sample ID");
+		checkArgument(startTime >= 0, "Negative start time");
+		checkArgument(endTime >= startTime, "End time before start time");
+
+		this.sampleId = sampleId;
+		this.startTime = startTime;
+		this.endTime = endTime;
+		this.stackTracesByTask = Collections.unmodifiableMap(stackTracesByTask);
+	}
+
+	/**
+	 * Returns the ID of the sample.
+	 *
+	 * @return ID of the sample
+	 */
+	public int getSampleId() {
+		return sampleId;
+	}
+
+	/**
+	 * Returns the time stamp, when the sample was triggered.
+	 *
+	 * @return Time stamp, when the sample was triggered
+	 */
+	public long getStartTime() {
+		return startTime;
+	}
+
+	/**
+	 * Returns the time stamp, when all stack traces were collected at the
+	 * JobManager.
+	 *
+	 * @return Time stamp, when all stack traces were collected at the
+	 * JobManager
+	 */
+	public long getEndTime() {
+		return endTime;
+	}
+
+	/**
+	 * Returns the a map of stack traces by execution ID.
+	 *
+	 * @return Map of stack traces by execution ID
+	 */
+	public Map<ExecutionAttemptID, List<StackTraceElement[]>> getStackTraces() {
+		return stackTracesByTask;
+	}
+
+	@Override
+	public String toString() {
+		return "StackTraceSample{" +
+				"sampleId=" + sampleId +
+				", startTime=" + startTime +
+				", endTime=" + endTime +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java
new file mode 100644
index 0000000..8c2ec6e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java
@@ -0,0 +1,392 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.messages.StackTraceSampleResponse;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A coordinator for triggering and collecting stack traces of running tasks.
+ */
+public class StackTraceSampleCoordinator {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StackTraceSampleCoordinator.class);
+
+	private static final int NUM_GHOST_SAMPLE_IDS = 10;
+
+	private final Object lock = new Object();
+
+	/** Executor used to run the futures. */
+	private final Executor executor;
+
+	/** Time out after the expected sampling duration. */
+	private final long sampleTimeout;
+
+	/** In progress samples (guarded by lock). */
+	private final Map<Integer, PendingStackTraceSample> pendingSamples = new HashMap<>();
+
+	/** A list of recent sample IDs to identify late messages vs. invalid ones. */
+	private final ArrayDeque<Integer> recentPendingSamples = new ArrayDeque<>(NUM_GHOST_SAMPLE_IDS);
+
+	/** Sample ID counter (guarded by lock). */
+	private int sampleIdCounter;
+
+	/**
+	 * Flag indicating whether the coordinator is still running (guarded by
+	 * lock).
+	 */
+	private boolean isShutDown;
+
+	/**
+	 * Creates a new coordinator for the job.
+	 *
+	 * @param executor to use to execute the futures
+	 * @param sampleTimeout Time out after the expected sampling duration.
+	 *                      This is added to the expected duration of a
+	 *                      sample, which is determined by the number of
+	 *                      samples and the delay between each sample.
+	 */
+	public StackTraceSampleCoordinator(Executor executor, long sampleTimeout) {
+		checkArgument(sampleTimeout >= 0L);
+		this.executor = Preconditions.checkNotNull(executor);
+		this.sampleTimeout = sampleTimeout;
+	}
+
+	/**
+	 * Triggers a stack trace sample to all tasks.
+	 *
+	 * @param tasksToSample       Tasks to sample.
+	 * @param numSamples          Number of stack trace samples to collect.
+	 * @param delayBetweenSamples Delay between consecutive samples.
+	 * @param maxStackTraceDepth  Maximum depth of the stack trace. 0 indicates
+	 *                            no maximum and keeps the complete stack trace.
+	 * @return A future of the completed stack trace sample
+	 */
+	@SuppressWarnings("unchecked")
+	public CompletableFuture<StackTraceSample> triggerStackTraceSample(
+			ExecutionVertex[] tasksToSample,
+			int numSamples,
+			Time delayBetweenSamples,
+			int maxStackTraceDepth) {
+
+		checkNotNull(tasksToSample, "Tasks to sample");
+		checkArgument(tasksToSample.length >= 1, "No tasks to sample");
+		checkArgument(numSamples >= 1, "No number of samples");
+		checkArgument(maxStackTraceDepth >= 0, "Negative maximum stack trace depth");
+
+		// Execution IDs of running tasks
+		ExecutionAttemptID[] triggerIds = new ExecutionAttemptID[tasksToSample.length];
+		Execution[] executions = new Execution[tasksToSample.length];
+
+		// Check that all tasks are RUNNING before triggering anything. The
+		// triggering can still fail.
+		for (int i = 0; i < triggerIds.length; i++) {
+			Execution execution = tasksToSample[i].getCurrentExecutionAttempt();
+			if (execution != null && execution.getState() == ExecutionState.RUNNING) {
+				executions[i] = execution;
+				triggerIds[i] = execution.getAttemptId();
+			} else {
+				return FutureUtils.completedExceptionally(new IllegalStateException("Task " + tasksToSample[i]
+					.getTaskNameWithSubtaskIndex() + " is not running."));
+			}
+		}
+
+		synchronized (lock) {
+			if (isShutDown) {
+				return FutureUtils.completedExceptionally(new IllegalStateException("Shut down"));
+			}
+
+			final int sampleId = sampleIdCounter++;
+
+			LOG.debug("Triggering stack trace sample {}", sampleId);
+
+			final PendingStackTraceSample pending = new PendingStackTraceSample(
+					sampleId, triggerIds);
+
+			// Discard the sample if it takes too long. We don't send cancel
+			// messages to the task managers, but only wait for the responses
+			// and then ignore them.
+			long expectedDuration = numSamples * delayBetweenSamples.toMilliseconds();
+			Time timeout = Time.milliseconds(expectedDuration + sampleTimeout);
+
+			// Add the pending sample before scheduling the discard task to
+			// prevent races with removing it again.
+			pendingSamples.put(sampleId, pending);
+
+			// Trigger all samples
+			for (Execution execution: executions) {
+				final CompletableFuture<StackTraceSampleResponse> stackTraceSampleFuture = execution.requestStackTraceSample(
+					sampleId,
+					numSamples,
+					delayBetweenSamples,
+					maxStackTraceDepth,
+					timeout);
+
+				stackTraceSampleFuture.handleAsync(
+					(StackTraceSampleResponse stackTraceSampleResponse, Throwable throwable) -> {
+						if (stackTraceSampleResponse != null) {
+							collectStackTraces(
+								stackTraceSampleResponse.getSampleId(),
+								stackTraceSampleResponse.getExecutionAttemptID(),
+								stackTraceSampleResponse.getSamples());
+						} else {
+							cancelStackTraceSample(sampleId, throwable);
+						}
+
+						return null;
+					},
+					executor);
+			}
+
+			return pending.getStackTraceSampleFuture();
+		}
+	}
+
+	/**
+	 * Cancels a pending sample.
+	 *
+	 * @param sampleId ID of the sample to cancel.
+	 * @param cause Cause of the cancelling (can be <code>null</code>).
+	 */
+	public void cancelStackTraceSample(int sampleId, Throwable cause) {
+		synchronized (lock) {
+			if (isShutDown) {
+				return;
+			}
+
+			PendingStackTraceSample sample = pendingSamples.remove(sampleId);
+			if (sample != null) {
+				if (cause != null) {
+					LOG.info("Cancelling sample " + sampleId, cause);
+				} else {
+					LOG.info("Cancelling sample {}", sampleId);
+				}
+
+				sample.discard(cause);
+				rememberRecentSampleId(sampleId);
+			}
+		}
+	}
+
+	/**
+	 * Shuts down the coordinator.
+	 *
+	 * <p>After shut down, no further operations are executed.
+	 */
+	public void shutDown() {
+		synchronized (lock) {
+			if (!isShutDown) {
+				LOG.info("Shutting down stack trace sample coordinator.");
+
+				for (PendingStackTraceSample pending : pendingSamples.values()) {
+					pending.discard(new RuntimeException("Shut down"));
+				}
+
+				pendingSamples.clear();
+
+				isShutDown = true;
+			}
+		}
+	}
+
+	/**
+	 * Collects stack traces of a task.
+	 *
+	 * @param sampleId    ID of the sample.
+	 * @param executionId ID of the sampled task.
+	 * @param stackTraces Stack traces of the sampled task.
+	 *
+	 * @throws IllegalStateException If unknown sample ID and not recently
+	 *                               finished or cancelled sample.
+	 */
+	public void collectStackTraces(
+			int sampleId,
+			ExecutionAttemptID executionId,
+			List<StackTraceElement[]> stackTraces) {
+
+		synchronized (lock) {
+			if (isShutDown) {
+				return;
+			}
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Collecting stack trace sample {} of task {}", sampleId, executionId);
+			}
+
+			PendingStackTraceSample pending = pendingSamples.get(sampleId);
+
+			if (pending != null) {
+				pending.collectStackTraces(executionId, stackTraces);
+
+				// Publish the sample
+				if (pending.isComplete()) {
+					pendingSamples.remove(sampleId);
+					rememberRecentSampleId(sampleId);
+
+					pending.completePromiseAndDiscard();
+				}
+			} else if (recentPendingSamples.contains(sampleId)) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Received late stack trace sample {} of task {}",
+							sampleId, executionId);
+				}
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Unknown sample ID " + sampleId);
+				}
+			}
+		}
+	}
+
+	private void rememberRecentSampleId(int sampleId) {
+		if (recentPendingSamples.size() >= NUM_GHOST_SAMPLE_IDS) {
+			recentPendingSamples.removeFirst();
+		}
+		recentPendingSamples.addLast(sampleId);
+	}
+
+	int getNumberOfPendingSamples() {
+		synchronized (lock) {
+			return pendingSamples.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A pending stack trace sample, which collects stack traces and owns a
+	 * {@link StackTraceSample} promise.
+	 *
+	 * <p>Access pending sample in lock scope.
+	 */
+	private static class PendingStackTraceSample {
+
+		private final int sampleId;
+		private final long startTime;
+		private final Set<ExecutionAttemptID> pendingTasks;
+		private final Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask;
+		private final CompletableFuture<StackTraceSample> stackTraceFuture;
+
+		private boolean isDiscarded;
+
+		PendingStackTraceSample(
+				int sampleId,
+				ExecutionAttemptID[] tasksToCollect) {
+
+			this.sampleId = sampleId;
+			this.startTime = System.currentTimeMillis();
+			this.pendingTasks = new HashSet<>(Arrays.asList(tasksToCollect));
+			this.stackTracesByTask = Maps.newHashMapWithExpectedSize(tasksToCollect.length);
+			this.stackTraceFuture = new CompletableFuture<>();
+		}
+
+		int getSampleId() {
+			return sampleId;
+		}
+
+		long getStartTime() {
+			return startTime;
+		}
+
+		boolean isDiscarded() {
+			return isDiscarded;
+		}
+
+		boolean isComplete() {
+			if (isDiscarded) {
+				throw new IllegalStateException("Discarded");
+			}
+
+			return pendingTasks.isEmpty();
+		}
+
+		void discard(Throwable cause) {
+			if (!isDiscarded) {
+				pendingTasks.clear();
+				stackTracesByTask.clear();
+
+				stackTraceFuture.completeExceptionally(new RuntimeException("Discarded", cause));
+
+				isDiscarded = true;
+			}
+		}
+
+		void collectStackTraces(ExecutionAttemptID executionId, List<StackTraceElement[]> stackTraces) {
+			if (isDiscarded) {
+				throw new IllegalStateException("Discarded");
+			}
+
+			if (pendingTasks.remove(executionId)) {
+				stackTracesByTask.put(executionId, Collections.unmodifiableList(stackTraces));
+			} else if (isComplete()) {
+				throw new IllegalStateException("Completed");
+			} else {
+				throw new IllegalArgumentException("Unknown task " + executionId);
+			}
+		}
+
+		void completePromiseAndDiscard() {
+			if (isComplete()) {
+				isDiscarded = true;
+
+				long endTime = System.currentTimeMillis();
+
+				StackTraceSample stackTraceSample = new StackTraceSample(
+						sampleId,
+						startTime,
+						endTime,
+						stackTracesByTask);
+
+				stackTraceFuture.complete(stackTraceSample);
+			} else {
+				throw new IllegalStateException("Not completed yet");
+			}
+		}
+
+		@SuppressWarnings("unchecked")
+		CompletableFuture<StackTraceSample> getStackTraceSampleFuture() {
+			return stackTraceFuture;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
new file mode 100644
index 0000000..2086628
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointConfigHandler.java
@@ -0,0 +1,120 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
+import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractExecutionGraphRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Handler that returns a job's snapshotting settings.
+ */
+public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String CHECKPOINT_CONFIG_REST_PATH = "/jobs/:jobid/checkpoints/config";
+
+	public CheckpointConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CHECKPOINT_CONFIG_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createCheckpointConfigJson(graph);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create checkpoint config json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the CheckpointConfigHandler.
+	 */
+	public static class CheckpointConfigJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createCheckpointConfigJson(graph);
+			String path = CHECKPOINT_CONFIG_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+
+	private static String createCheckpointConfigJson(AccessExecutionGraph graph) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+		JobCheckpointingSettings settings = graph.getJobCheckpointingSettings();
+
+		if (settings == null) {
+			return "{}";
+		}
+
+		gen.writeStartObject();
+		{
+			gen.writeStringField("mode", settings.isExactlyOnce() ? "exactly_once" : "at_least_once");
+			gen.writeNumberField("interval", settings.getCheckpointInterval());
+			gen.writeNumberField("timeout", settings.getCheckpointTimeout());
+			gen.writeNumberField("min_pause", settings.getMinPauseBetweenCheckpoints());
+			gen.writeNumberField("max_concurrent", settings.getMaxConcurrentCheckpoints());
+
+			ExternalizedCheckpointSettings externalization = settings.getExternalizedCheckpointSettings();
+			gen.writeObjectFieldStart("externalization");
+			{
+				if (externalization.externalizeCheckpoints()) {
+					gen.writeBooleanField("enabled", true);
+					gen.writeBooleanField("delete_on_cancellation", externalization.deleteOnCancellation());
+				} else {
+					gen.writeBooleanField("enabled", false);
+				}
+			}
+			gen.writeEndObject();
+
+		}
+		gen.writeEndObject();
+
+		gen.close();
+
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCache.java
new file mode 100644
index 0000000..f21fc76
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCache.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import javax.annotation.Nullable;
+
+/**
+ * A size-based cache of accessed checkpoints for completed and failed
+ * checkpoints.
+ *
+ * <p>Having this cache in place for accessed stats improves the user
+ * experience quite a bit as accessed checkpoint stats stay available
+ * and don't expire. For example if you manage to click on the last
+ * checkpoint in the history, it is not available via the stats as soon
+ * as another checkpoint is triggered. With the cache in place, the
+ * checkpoint will still be available for investigation.
+ */
+public class CheckpointStatsCache {
+
+	@Nullable
+	private final Cache<Long, AbstractCheckpointStats> cache;
+
+	public CheckpointStatsCache(int maxNumEntries) {
+		if (maxNumEntries > 0) {
+			this.cache = CacheBuilder.<Long, AbstractCheckpointStats>newBuilder()
+				.maximumSize(maxNumEntries)
+				.build();
+		} else {
+			this.cache = null;
+		}
+	}
+
+	/**
+	 * Try to add the checkpoint to the cache.
+	 *
+	 * @param checkpoint Checkpoint to be added.
+	 */
+	void tryAdd(AbstractCheckpointStats checkpoint) {
+		// Don't add in progress checkpoints as they will be replaced by their
+		// completed/failed version eventually.
+		if (cache != null && checkpoint != null && !checkpoint.getStatus().isInProgress()) {
+			cache.put(checkpoint.getCheckpointId(), checkpoint);
+		}
+	}
+
+	/**
+	 * Try to look up a checkpoint by it's ID in the cache.
+	 *
+	 * @param checkpointId ID of the checkpoint to look up.
+	 * @return The checkpoint or <code>null</code> if checkpoint not found.
+	 */
+	AbstractCheckpointStats tryGet(long checkpointId) {
+		if (cache != null) {
+			return cache.getIfPresent(checkpointId);
+		} else {
+			return null;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandler.java
new file mode 100644
index 0000000..61ebeda
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsHandler.java
@@ -0,0 +1,203 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.TaskStateStats;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractExecutionGraphRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns checkpoint stats for a single job vertex.
+ */
+public class CheckpointStatsDetailsHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String CHECKPOINT_STATS_DETAILS_REST_PATH = "/jobs/:jobid/checkpoints/details/:checkpointid";
+
+	private final CheckpointStatsCache cache;
+
+	public CheckpointStatsDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) {
+		super(executionGraphHolder, executor);
+		this.cache = cache;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CHECKPOINT_STATS_DETAILS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				long checkpointId = parseCheckpointId(params);
+				if (checkpointId == -1) {
+					return "{}";
+				}
+
+				CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
+				if (snapshot == null) {
+					return "{}";
+				}
+
+				AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId);
+
+				if (checkpoint != null) {
+					cache.tryAdd(checkpoint);
+				} else {
+					checkpoint = cache.tryGet(checkpointId);
+
+					if (checkpoint == null) {
+						return "{}";
+					}
+				}
+
+				try {
+					return createCheckpointDetailsJson(checkpoint);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create checkpoint details json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the CheckpointStatsDetails.
+	 */
+	public static class CheckpointStatsDetailsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			CheckpointStatsSnapshot stats = graph.getCheckpointStatsSnapshot();
+			if (stats == null) {
+				return Collections.emptyList();
+			}
+			CheckpointStatsHistory history = stats.getHistory();
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
+				String json = createCheckpointDetailsJson(checkpoint);
+				String path = CHECKPOINT_STATS_DETAILS_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":checkpointid", String.valueOf(checkpoint.getCheckpointId()));
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createCheckpointDetailsJson(AbstractCheckpointStats checkpoint) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+		gen.writeStartObject();
+
+		gen.writeNumberField("id", checkpoint.getCheckpointId());
+		gen.writeStringField("status", checkpoint.getStatus().toString());
+		gen.writeBooleanField("is_savepoint", checkpoint.getProperties().isSavepoint());
+		gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
+		gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
+		gen.writeNumberField("state_size", checkpoint.getStateSize());
+		gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
+		gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
+		gen.writeNumberField("num_subtasks", checkpoint.getNumberOfSubtasks());
+		gen.writeNumberField("num_acknowledged_subtasks", checkpoint.getNumberOfAcknowledgedSubtasks());
+
+		if (checkpoint.getStatus().isCompleted()) {
+			// --- Completed ---
+			CompletedCheckpointStats completed = (CompletedCheckpointStats) checkpoint;
+
+			String externalPath = completed.getExternalPath();
+			if (externalPath != null) {
+				gen.writeStringField("external_path", externalPath);
+			}
+
+			gen.writeBooleanField("discarded", completed.isDiscarded());
+		}
+		else if (checkpoint.getStatus().isFailed()) {
+			// --- Failed ---
+			FailedCheckpointStats failed = (FailedCheckpointStats) checkpoint;
+
+			gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
+
+			String failureMsg = failed.getFailureMessage();
+			if (failureMsg != null) {
+				gen.writeStringField("failure_message", failureMsg);
+			}
+		}
+
+		gen.writeObjectFieldStart("tasks");
+		for (TaskStateStats taskStats : checkpoint.getAllTaskStateStats()) {
+			gen.writeObjectFieldStart(taskStats.getJobVertexId().toString());
+
+			gen.writeNumberField("latest_ack_timestamp", taskStats.getLatestAckTimestamp());
+			gen.writeNumberField("state_size", taskStats.getStateSize());
+			gen.writeNumberField("end_to_end_duration", taskStats.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
+			gen.writeNumberField("alignment_buffered", taskStats.getAlignmentBuffered());
+			gen.writeNumberField("num_subtasks", taskStats.getNumberOfSubtasks());
+			gen.writeNumberField("num_acknowledged_subtasks", taskStats.getNumberOfAcknowledgedSubtasks());
+
+			gen.writeEndObject();
+		}
+		gen.writeEndObject();
+
+		gen.writeEndObject();
+		gen.close();
+
+		return writer.toString();
+	}
+
+	/**
+	 * Returns the checkpoint ID parsed from the provided parameters.
+	 *
+	 * @param params Path parameters
+	 * @return Parsed checkpoint ID or <code>-1</code> if not available.
+	 */
+	static long parseCheckpointId(Map<String, String> params) {
+		String param = params.get("checkpointid");
+		if (param == null) {
+			return -1;
+		}
+
+		try {
+			return Long.parseLong(param);
+		} catch (NumberFormatException ignored) {
+			return -1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsSubtasksHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
new file mode 100644
index 0000000..22a8db2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
@@ -0,0 +1,233 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
+import org.apache.flink.runtime.checkpoint.SubtaskStateStats;
+import org.apache.flink.runtime.checkpoint.TaskStateStats;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractExecutionGraphRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJobVertexRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Request handler that returns checkpoint stats for a single job vertex with
+ * the summary stats and all subtasks.
+ */
+public class CheckpointStatsDetailsSubtasksHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH = "/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid";
+
+	private final CheckpointStatsCache cache;
+
+	public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) {
+		super(executionGraphHolder, executor);
+		this.cache = checkNotNull(cache);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(
+			Map<String, String> pathParams,
+			Map<String, String> queryParams,
+			JobManagerGateway jobManagerGateway) {
+		return super.handleJsonRequest(pathParams, queryParams, jobManagerGateway);
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		long checkpointId = CheckpointStatsDetailsHandler.parseCheckpointId(params);
+		if (checkpointId == -1) {
+			return CompletableFuture.completedFuture("{}");
+		}
+
+		JobVertexID vertexId = AbstractJobVertexRequestHandler.parseJobVertexId(params);
+		if (vertexId == null) {
+			return CompletableFuture.completedFuture("{}");
+		}
+
+		CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
+		if (snapshot == null) {
+			return CompletableFuture.completedFuture("{}");
+		}
+
+		AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId);
+
+		if (checkpoint != null) {
+			cache.tryAdd(checkpoint);
+		} else {
+			checkpoint = cache.tryGet(checkpointId);
+
+			if (checkpoint == null) {
+				return CompletableFuture.completedFuture("{}");
+			}
+		}
+
+		TaskStateStats taskStats = checkpoint.getTaskStateStats(vertexId);
+		if (taskStats == null) {
+			return CompletableFuture.completedFuture("{}");
+		}
+
+		try {
+			return CompletableFuture.completedFuture(createSubtaskCheckpointDetailsJson(checkpoint, taskStats));
+		} catch (IOException e) {
+			return FutureUtils.completedExceptionally(e);
+		}
+	}
+
+	/**
+	 * Archivist for the CheckpointStatsDetailsSubtasksHandler.
+	 */
+	public static class CheckpointStatsDetailsSubtasksJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			CheckpointStatsSnapshot stats = graph.getCheckpointStatsSnapshot();
+			if (stats == null) {
+				return Collections.emptyList();
+			}
+			CheckpointStatsHistory history = stats.getHistory();
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
+				for (TaskStateStats subtaskStats : checkpoint.getAllTaskStateStats()) {
+					String json = createSubtaskCheckpointDetailsJson(checkpoint, subtaskStats);
+					String path = CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH
+						.replace(":jobid", graph.getJobID().toString())
+						.replace(":checkpointid", String.valueOf(checkpoint.getCheckpointId()))
+						.replace(":vertexid", subtaskStats.getJobVertexId().toString());
+					archive.add(new ArchivedJson(path, json));
+				}
+			}
+			return archive;
+		}
+	}
+
+	private static String createSubtaskCheckpointDetailsJson(AbstractCheckpointStats checkpoint, TaskStateStats taskStats) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+		// Overview
+		gen.writeNumberField("id", checkpoint.getCheckpointId());
+		gen.writeStringField("status", checkpoint.getStatus().toString());
+		gen.writeNumberField("latest_ack_timestamp", taskStats.getLatestAckTimestamp());
+		gen.writeNumberField("state_size", taskStats.getStateSize());
+		gen.writeNumberField("end_to_end_duration", taskStats.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
+		gen.writeNumberField("alignment_buffered", taskStats.getAlignmentBuffered());
+		gen.writeNumberField("num_subtasks", taskStats.getNumberOfSubtasks());
+		gen.writeNumberField("num_acknowledged_subtasks", taskStats.getNumberOfAcknowledgedSubtasks());
+
+		if (taskStats.getNumberOfAcknowledgedSubtasks() > 0) {
+			gen.writeObjectFieldStart("summary");
+			gen.writeObjectFieldStart("state_size");
+			CheckpointStatsHandler.writeMinMaxAvg(gen, taskStats.getSummaryStats().getStateSizeStats());
+			gen.writeEndObject();
+
+			gen.writeObjectFieldStart("end_to_end_duration");
+			MinMaxAvgStats ackTimestampStats = taskStats.getSummaryStats().getAckTimestampStats();
+			gen.writeNumberField("min", Math.max(0, ackTimestampStats.getMinimum() - checkpoint.getTriggerTimestamp()));
+			gen.writeNumberField("max", Math.max(0, ackTimestampStats.getMaximum() - checkpoint.getTriggerTimestamp()));
+			gen.writeNumberField("avg", Math.max(0, ackTimestampStats.getAverage() - checkpoint.getTriggerTimestamp()));
+			gen.writeEndObject();
+
+			gen.writeObjectFieldStart("checkpoint_duration");
+			gen.writeObjectFieldStart("sync");
+			CheckpointStatsHandler.writeMinMaxAvg(gen, taskStats.getSummaryStats().getSyncCheckpointDurationStats());
+			gen.writeEndObject();
+			gen.writeObjectFieldStart("async");
+			CheckpointStatsHandler.writeMinMaxAvg(gen, taskStats.getSummaryStats().getAsyncCheckpointDurationStats());
+			gen.writeEndObject();
+			gen.writeEndObject();
+
+			gen.writeObjectFieldStart("alignment");
+			gen.writeObjectFieldStart("buffered");
+			CheckpointStatsHandler.writeMinMaxAvg(gen, taskStats.getSummaryStats().getAlignmentBufferedStats());
+			gen.writeEndObject();
+			gen.writeObjectFieldStart("duration");
+			CheckpointStatsHandler.writeMinMaxAvg(gen, taskStats.getSummaryStats().getAlignmentDurationStats());
+			gen.writeEndObject();
+			gen.writeEndObject();
+			gen.writeEndObject();
+		}
+
+		SubtaskStateStats[] subtasks = taskStats.getSubtaskStats();
+
+		gen.writeArrayFieldStart("subtasks");
+		for (int i = 0; i < subtasks.length; i++) {
+			SubtaskStateStats subtask = subtasks[i];
+
+			gen.writeStartObject();
+			gen.writeNumberField("index", i);
+
+			if (subtask != null) {
+				gen.writeStringField("status", "completed");
+				gen.writeNumberField("ack_timestamp", subtask.getAckTimestamp());
+				gen.writeNumberField("end_to_end_duration", subtask.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
+				gen.writeNumberField("state_size", subtask.getStateSize());
+
+				gen.writeObjectFieldStart("checkpoint");
+				gen.writeNumberField("sync", subtask.getSyncCheckpointDuration());
+				gen.writeNumberField("async", subtask.getAsyncCheckpointDuration());
+				gen.writeEndObject();
+
+				gen.writeObjectFieldStart("alignment");
+				gen.writeNumberField("buffered", subtask.getAlignmentBuffered());
+				gen.writeNumberField("duration", subtask.getAlignmentDuration());
+				gen.writeEndObject();
+			} else {
+				gen.writeStringField("status", "pending_or_failed");
+			}
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+		gen.close();
+
+		return writer.toString();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandler.java
new file mode 100644
index 0000000..abb353e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsHandler.java
@@ -0,0 +1,277 @@
+/*
+ * 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.rest.handler.legacy.checkpoints;
+
+import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStatsSummary;
+import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
+import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
+import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractExecutionGraphRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphHolder;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Handler that returns checkpoint statistics for a job.
+ */
+public class CheckpointStatsHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String CHECKPOINT_STATS_REST_PATH = "/jobs/:jobid/checkpoints";
+
+	public CheckpointStatsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{CHECKPOINT_STATS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createCheckpointStatsJson(graph);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create checkpoint stats json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the CheckpointStatsJsonHandler.
+	 */
+	public static class CheckpointStatsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createCheckpointStatsJson(graph);
+			String path = CHECKPOINT_STATS_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+
+	private static String createCheckpointStatsJson(AccessExecutionGraph graph) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
+		if (snapshot == null) {
+			return "{}";
+		}
+
+		gen.writeStartObject();
+
+		// Counts
+		writeCounts(gen, snapshot.getCounts());
+
+		// Summary
+		writeSummary(gen, snapshot.getSummaryStats());
+
+		CheckpointStatsHistory history = snapshot.getHistory();
+
+		// Latest
+		writeLatestCheckpoints(
+			gen,
+			history.getLatestCompletedCheckpoint(),
+			history.getLatestSavepoint(),
+			history.getLatestFailedCheckpoint(),
+			snapshot.getLatestRestoredCheckpoint());
+
+		// History
+		writeHistory(gen, snapshot.getHistory());
+
+		gen.writeEndObject();
+		gen.close();
+
+		return writer.toString();
+	}
+
+	private static void writeCounts(JsonGenerator gen, CheckpointStatsCounts counts) throws IOException {
+		gen.writeObjectFieldStart("counts");
+		gen.writeNumberField("restored", counts.getNumberOfRestoredCheckpoints());
+		gen.writeNumberField("total", counts.getTotalNumberOfCheckpoints());
+		gen.writeNumberField("in_progress", counts.getNumberOfInProgressCheckpoints());
+		gen.writeNumberField("completed", counts.getNumberOfCompletedCheckpoints());
+		gen.writeNumberField("failed", counts.getNumberOfFailedCheckpoints());
+		gen.writeEndObject();
+	}
+
+	private static void writeSummary(
+		JsonGenerator gen,
+		CompletedCheckpointStatsSummary summary) throws IOException {
+		gen.writeObjectFieldStart("summary");
+		gen.writeObjectFieldStart("state_size");
+		writeMinMaxAvg(gen, summary.getStateSizeStats());
+		gen.writeEndObject();
+
+		gen.writeObjectFieldStart("end_to_end_duration");
+		writeMinMaxAvg(gen, summary.getEndToEndDurationStats());
+		gen.writeEndObject();
+
+		gen.writeObjectFieldStart("alignment_buffered");
+		writeMinMaxAvg(gen, summary.getAlignmentBufferedStats());
+		gen.writeEndObject();
+		gen.writeEndObject();
+	}
+
+	static void writeMinMaxAvg(JsonGenerator gen, MinMaxAvgStats minMaxAvg) throws IOException {
+		gen.writeNumberField("min", minMaxAvg.getMinimum());
+		gen.writeNumberField("max", minMaxAvg.getMaximum());
+		gen.writeNumberField("avg", minMaxAvg.getAverage());
+	}
+
+	private static void writeLatestCheckpoints(
+		JsonGenerator gen,
+		@Nullable CompletedCheckpointStats completed,
+		@Nullable CompletedCheckpointStats savepoint,
+		@Nullable FailedCheckpointStats failed,
+		@Nullable RestoredCheckpointStats restored) throws IOException {
+
+		gen.writeObjectFieldStart("latest");
+		// Completed checkpoint
+		if (completed != null) {
+			gen.writeObjectFieldStart("completed");
+			writeCheckpoint(gen, completed);
+
+			String externalPath = completed.getExternalPath();
+			if (externalPath != null) {
+				gen.writeStringField("external_path", completed.getExternalPath());
+			}
+
+			gen.writeEndObject();
+		}
+
+		// Completed savepoint
+		if (savepoint != null) {
+			gen.writeObjectFieldStart("savepoint");
+			writeCheckpoint(gen, savepoint);
+
+			String externalPath = savepoint.getExternalPath();
+			if (externalPath != null) {
+				gen.writeStringField("external_path", savepoint.getExternalPath());
+			}
+			gen.writeEndObject();
+		}
+
+		// Failed checkpoint
+		if (failed != null) {
+			gen.writeObjectFieldStart("failed");
+			writeCheckpoint(gen, failed);
+
+			gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
+			String failureMsg = failed.getFailureMessage();
+			if (failureMsg != null) {
+				gen.writeStringField("failure_message", failureMsg);
+			}
+			gen.writeEndObject();
+		}
+
+		// Restored checkpoint
+		if (restored != null) {
+			gen.writeObjectFieldStart("restored");
+			gen.writeNumberField("id", restored.getCheckpointId());
+			gen.writeNumberField("restore_timestamp", restored.getRestoreTimestamp());
+			gen.writeBooleanField("is_savepoint", restored.getProperties().isSavepoint());
+
+			String externalPath = restored.getExternalPath();
+			if (externalPath != null) {
+				gen.writeStringField("external_path", externalPath);
+			}
+			gen.writeEndObject();
+		}
+		gen.writeEndObject();
+	}
+
+	private static void writeCheckpoint(JsonGenerator gen, AbstractCheckpointStats checkpoint) throws IOException {
+		gen.writeNumberField("id", checkpoint.getCheckpointId());
+		gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
+		gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
+		gen.writeNumberField("state_size", checkpoint.getStateSize());
+		gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
+		gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
+
+	}
+
+	private static void writeHistory(JsonGenerator gen, CheckpointStatsHistory history) throws IOException {
+		gen.writeArrayFieldStart("history");
+		for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
+			gen.writeStartObject();
+			gen.writeNumberField("id", checkpoint.getCheckpointId());
+			gen.writeStringField("status", checkpoint.getStatus().toString());
+			gen.writeBooleanField("is_savepoint", checkpoint.getProperties().isSavepoint());
+			gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
+			gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
+			gen.writeNumberField("state_size", checkpoint.getStateSize());
+			gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
+			gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
+			gen.writeNumberField("num_subtasks", checkpoint.getNumberOfSubtasks());
+			gen.writeNumberField("num_acknowledged_subtasks", checkpoint.getNumberOfAcknowledgedSubtasks());
+
+			if (checkpoint.getStatus().isCompleted()) {
+				// --- Completed ---
+				CompletedCheckpointStats completed = (CompletedCheckpointStats) checkpoint;
+
+				String externalPath = completed.getExternalPath();
+				if (externalPath != null) {
+					gen.writeStringField("external_path", externalPath);
+				}
+
+				gen.writeBooleanField("discarded", completed.isDiscarded());
+			}
+			else if (checkpoint.getStatus().isFailed()) {
+				// --- Failed ---
+				FailedCheckpointStats failed = (FailedCheckpointStats) checkpoint;
+
+				gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
+
+				String failureMsg = failed.getFailureMessage();
+				if (failureMsg != null) {
+					gen.writeStringField("failure_message", failureMsg);
+				}
+			}
+
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+	}
+}


[09/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
deleted file mode 100644
index 9c5e168..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java
+++ /dev/null
@@ -1,389 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
-import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
-import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
-import org.apache.flink.runtime.checkpoint.SubtaskStateStats;
-import org.apache.flink.runtime.checkpoint.TaskStateStats;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the CheckpointStatsSubtaskDetailsHandler.
- */
-public class CheckpointStatsSubtaskDetailsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new CheckpointStatsDetailsSubtasksHandler.CheckpointStatsDetailsSubtasksJsonArchivist();
-		ObjectMapper mapper = new ObjectMapper();
-
-		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
-
-		TaskStateStats task = createTaskStateStats(1237);
-		when(checkpoint.getAllTaskStateStats()).thenReturn(Collections.singletonList(task));
-
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpoints()).thenReturn(Collections.<AbstractCheckpointStats>singletonList(checkpoint));
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-		when(graph.getJobID()).thenReturn(new JobID());
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals(
-			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + checkpoint.getCheckpointId() + "/subtasks/" + task.getJobVertexId(),
-			archive.getPath());
-		JsonNode rootNode = mapper.readTree(archive.getJson());
-		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
-		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
-
-		verifyTaskNode(rootNode, task, checkpoint.getTriggerTimestamp());
-	}
-
-	@Test
-	public void testGetPaths() {
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", paths[0]);
-	}
-
-	/**
-	 * Tests a subtask details request.
-	 */
-	@Test
-	public void testSubtaskRequest() throws Exception {
-		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
-
-		TaskStateStats task = createTaskStateStats(1237);
-		when(checkpoint.getTaskStateStats(any(JobVertexID.class))).thenReturn(task);
-
-		JsonNode rootNode = triggerRequest(checkpoint);
-		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
-		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
-
-		verifyTaskNode(rootNode, task, checkpoint.getTriggerTimestamp());
-	}
-
-	/**
-	 * Tests a subtask details request.
-	 */
-	@Test
-	public void testSubtaskRequestNoSummary() throws Exception {
-		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(checkpoint.getTriggerTimestamp()).thenReturn(0L); // ack timestamp = duration
-
-		TaskStateStats task = createTaskStateStats(0); // no acknowledged
-		when(checkpoint.getTaskStateStats(any(JobVertexID.class))).thenReturn(task);
-
-		JsonNode rootNode = triggerRequest(checkpoint);
-		assertNull(rootNode.get("summary"));
-	}
-
-	/**
-	 * Tests request with illegal checkpoint ID param.
-	 */
-	@Test
-	public void testIllegalCheckpointId() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "illegal checkpoint");
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Tests request with missing checkpoint ID param.
-	 */
-	@Test
-	public void testNoCheckpointIdParam() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Test lookup of not existing checkpoint in history.
-	 */
-	@Test
-	public void testCheckpointNotFound() throws Exception {
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpointById(anyLong())).thenReturn(null); // not found
-
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "123");
-		params.put("vertexid", new JobVertexID().toString());
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-		verify(history, times(1)).getCheckpointById(anyLong());
-	}
-
-	/**
-	 * Tests request with illegal job vertex ID param.
-	 */
-	@Test
-	public void testIllegalJobVertexIdParam() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "1");
-		params.put("vertexid", "illegal vertex id");
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Tests request with missing job vertex ID param.
-	 */
-	@Test
-	public void testNoJobVertexIdParam() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "1");
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Test lookup of not existing job vertex ID in checkpoint.
-	 */
-	@Test
-	public void testJobVertexNotFound() throws Exception {
-		PendingCheckpointStats inProgress = mock(PendingCheckpointStats.class);
-		when(inProgress.getTaskStateStats(any(JobVertexID.class))).thenReturn(null); // not found
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpointById(anyLong())).thenReturn(inProgress);
-
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "123");
-		params.put("vertexid", new JobVertexID().toString());
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-		verify(inProgress, times(1)).getTaskStateStats(any(JobVertexID.class));
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throws Exception {
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpointById(anyLong())).thenReturn(checkpoint);
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "123");
-		params.put("vertexid", new JobVertexID().toString());
-		String json = handler.handleRequest(graph, params).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		return mapper.readTree(json);
-	}
-
-	private static TaskStateStats createTaskStateStats(int numAcknowledged) {
-		ThreadLocalRandom rand = ThreadLocalRandom.current();
-
-		TaskStateStats task = mock(TaskStateStats.class);
-		when(task.getJobVertexId()).thenReturn(new JobVertexID());
-		when(task.getLatestAckTimestamp()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getStateSize()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getAlignmentBuffered()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getNumberOfSubtasks()).thenReturn(rand.nextInt(1024) + 1);
-		when(task.getNumberOfAcknowledgedSubtasks()).thenReturn(numAcknowledged);
-
-		TaskStateStats.TaskStateStatsSummary summary = mock(TaskStateStats.TaskStateStatsSummary.class);
-
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getStateSizeStats();
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getAckTimestampStats();
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getAlignmentBufferedStats();
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getAlignmentDurationStats();
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getSyncCheckpointDurationStats();
-		doReturn(createMinMaxAvgStats(rand)).when(summary).getAsyncCheckpointDurationStats();
-
-		when(task.getSummaryStats()).thenReturn(summary);
-
-		SubtaskStateStats[] subtasks = new SubtaskStateStats[3];
-		subtasks[0] = createSubtaskStats(0, rand);
-		subtasks[1] = createSubtaskStats(1, rand);
-		subtasks[2] = null;
-
-		when(task.getSubtaskStats()).thenReturn(subtasks);
-
-		return task;
-	}
-
-	private static void verifyTaskNode(JsonNode taskNode, TaskStateStats task, long triggerTimestamp) {
-		long duration = ThreadLocalRandom.current().nextInt(128);
-
-		assertEquals(task.getLatestAckTimestamp(), taskNode.get("latest_ack_timestamp").asLong());
-		assertEquals(task.getStateSize(), taskNode.get("state_size").asLong());
-		assertEquals(task.getEndToEndDuration(task.getLatestAckTimestamp() - duration), taskNode.get("end_to_end_duration").asLong());
-		assertEquals(task.getAlignmentBuffered(), taskNode.get("alignment_buffered").asLong());
-		assertEquals(task.getNumberOfSubtasks(), taskNode.get("num_subtasks").asInt());
-		assertEquals(task.getNumberOfAcknowledgedSubtasks(), taskNode.get("num_acknowledged_subtasks").asInt());
-
-		TaskStateStats.TaskStateStatsSummary summary = task.getSummaryStats();
-		verifyMinMaxAvgStats(summary.getStateSizeStats(), taskNode.get("summary").get("state_size"));
-		verifyMinMaxAvgStats(summary.getSyncCheckpointDurationStats(), taskNode.get("summary").get("checkpoint_duration").get("sync"));
-		verifyMinMaxAvgStats(summary.getAsyncCheckpointDurationStats(), taskNode.get("summary").get("checkpoint_duration").get("async"));
-		verifyMinMaxAvgStats(summary.getAlignmentBufferedStats(), taskNode.get("summary").get("alignment").get("buffered"));
-		verifyMinMaxAvgStats(summary.getAlignmentDurationStats(), taskNode.get("summary").get("alignment").get("duration"));
-
-		JsonNode endToEndDurationNode = taskNode.get("summary").get("end_to_end_duration");
-		assertEquals(summary.getAckTimestampStats().getMinimum() - triggerTimestamp, endToEndDurationNode.get("min").asLong());
-		assertEquals(summary.getAckTimestampStats().getMaximum() - triggerTimestamp, endToEndDurationNode.get("max").asLong());
-		assertEquals((long) summary.getAckTimestampStats().getAverage() - triggerTimestamp, endToEndDurationNode.get("avg").asLong());
-
-		SubtaskStateStats[] subtasks = task.getSubtaskStats();
-		Iterator<JsonNode> it = taskNode.get("subtasks").iterator();
-
-		assertTrue(it.hasNext());
-		verifySubtaskStats(it.next(), 0, subtasks[0]);
-
-		assertTrue(it.hasNext());
-		verifySubtaskStats(it.next(), 1, subtasks[1]);
-
-		assertTrue(it.hasNext());
-		verifySubtaskStats(it.next(), 2, subtasks[2]);
-
-		assertFalse(it.hasNext());
-	}
-
-	private static SubtaskStateStats createSubtaskStats(int index, ThreadLocalRandom rand) {
-		SubtaskStateStats subtask = mock(SubtaskStateStats.class);
-		when(subtask.getSubtaskIndex()).thenReturn(index);
-		when(subtask.getAckTimestamp()).thenReturn(rand.nextLong(1024));
-		when(subtask.getAlignmentBuffered()).thenReturn(rand.nextLong(1024));
-		when(subtask.getAlignmentDuration()).thenReturn(rand.nextLong(1024));
-		when(subtask.getSyncCheckpointDuration()).thenReturn(rand.nextLong(1024));
-		when(subtask.getAsyncCheckpointDuration()).thenReturn(rand.nextLong(1024));
-		when(subtask.getAckTimestamp()).thenReturn(rand.nextLong(1024));
-		when(subtask.getStateSize()).thenReturn(rand.nextLong(1024));
-		when(subtask.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024));
-		return subtask;
-	}
-
-	private static void verifySubtaskStats(JsonNode subtaskNode, int index, SubtaskStateStats subtask) {
-		if (subtask == null) {
-			assertEquals(index, subtaskNode.get("index").asInt());
-			assertEquals("pending_or_failed", subtaskNode.get("status").asText());
-		} else {
-			assertEquals(subtask.getSubtaskIndex(), subtaskNode.get("index").asInt());
-			assertEquals("completed", subtaskNode.get("status").asText());
-			assertEquals(subtask.getAckTimestamp(), subtaskNode.get("ack_timestamp").asLong());
-			assertEquals(subtask.getEndToEndDuration(0), subtaskNode.get("end_to_end_duration").asLong());
-			assertEquals(subtask.getStateSize(), subtaskNode.get("state_size").asLong());
-			assertEquals(subtask.getSyncCheckpointDuration(), subtaskNode.get("checkpoint").get("sync").asLong());
-			assertEquals(subtask.getAsyncCheckpointDuration(), subtaskNode.get("checkpoint").get("async").asLong());
-			assertEquals(subtask.getAlignmentBuffered(), subtaskNode.get("alignment").get("buffered").asLong());
-			assertEquals(subtask.getAlignmentDuration(), subtaskNode.get("alignment").get("duration").asLong());
-		}
-	}
-
-	private static MinMaxAvgStats createMinMaxAvgStats(ThreadLocalRandom rand) {
-		MinMaxAvgStats mma = mock(MinMaxAvgStats.class);
-		when(mma.getMinimum()).thenReturn(rand.nextLong(1024));
-		when(mma.getMaximum()).thenReturn(rand.nextLong(1024));
-		when(mma.getAverage()).thenReturn(rand.nextLong(1024));
-
-		return mma;
-	}
-
-	private static void verifyMinMaxAvgStats(MinMaxAvgStats expected, JsonNode node) {
-		assertEquals(expected.getMinimum(), node.get("min").asLong());
-		assertEquals(expected.getMaximum(), node.get("max").asLong());
-		assertEquals(expected.getAverage(), node.get("avg").asLong());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/FsJobArchivistTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/FsJobArchivistTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/FsJobArchivistTest.java
index 2e52f2e..03666a8 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/FsJobArchivistTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/FsJobArchivistTest.java
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.webmonitor.history;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.history.FsJobArchivist;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
 import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
 
 import org.junit.Assert;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java
index 33d9c79..3c93be3 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.messages.ArchiveMessages;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java
deleted file mode 100644
index 0755888..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java
+++ /dev/null
@@ -1,172 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests for the AbstractMetricsHandler.
- */
-public class AbstractMetricsHandlerTest extends TestLogger {
-	/**
-	 * Verifies that the handlers correctly handle expected REST calls.
-	 */
-	@Test
-	public void testHandleRequest() throws Exception {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		Map<String, String> queryParams = new HashMap<>();
-
-		pathParams.put("jobid", "jobid");
-		pathParams.put("vertexid", "taskid");
-
-		// get list of available metrics
-		String availableList = handler.handleJsonRequest(pathParams, queryParams, null).get();
-
-		assertEquals("[" +
-				"{\"id\":\"8.opname.abc.metric5\"}," +
-				"{\"id\":\"8.abc.metric4\"}" +
-				"]",
-			availableList);
-
-		// get value for a single metric
-		queryParams.put("get", "8.opname.abc.metric5");
-
-		String metricValue = handler.handleJsonRequest(pathParams, queryParams, null).get();
-
-		assertEquals("[" +
-				"{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}" +
-				"]"
-			, metricValue
-		);
-
-		// get values for multiple metrics
-		queryParams.put("get", "8.opname.abc.metric5,8.abc.metric4");
-
-		String metricValues = handler.handleJsonRequest(pathParams, queryParams, null).get();
-
-		assertEquals("[" +
-				"{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}," +
-				"{\"id\":\"8.abc.metric4\",\"value\":\"3\"}" +
-				"]",
-			metricValues
-		);
-	}
-
-	/**
-	 * Verifies that a malformed request for available metrics does not throw an exception.
-	 */
-	@Test
-	public void testInvalidListDoesNotFail() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		Map<String, String> queryParams = new HashMap<>();
-
-		pathParams.put("jobid", "jobid");
-		pathParams.put("vertexid", "taskid");
-
-		//-----invalid variable
-		pathParams.put("jobid", "nonexistent");
-
-		try {
-			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
-		} catch (Exception e) {
-			fail();
-		}
-	}
-
-	/**
-	 * Verifies that a malformed request for a metric value does not throw an exception.
-	 */
-	@Test
-	public void testInvalidGetDoesNotFail() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		Map<String, String> queryParams = new HashMap<>();
-
-		pathParams.put("jobid", "jobid");
-		pathParams.put("vertexid", "taskid");
-
-		//-----empty string
-		queryParams.put("get", "");
-
-		try {
-			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
-		} catch (Exception e) {
-			fail(e.getMessage());
-		}
-
-		//-----invalid variable
-		pathParams.put("jobid", "nonexistent");
-		queryParams.put("get", "subindex.opname.abc.metric5");
-
-		try {
-			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
-		} catch (Exception e) {
-			fail(e.getMessage());
-		}
-
-		//-----invalid metric
-		pathParams.put("jobid", "nonexistant");
-		queryParams.put("get", "subindex.opname.abc.nonexistant");
-
-		try {
-			assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get());
-		} catch (Exception e) {
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java
deleted file mode 100644
index 6d17b40..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java
+++ /dev/null
@@ -1,84 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests for the JobManagerMetricsHandler.
- */
-public class JobManagerMetricsHandlerTest extends TestLogger {
-	@Test
-	public void testGetPaths() {
-		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobmanager/metrics", paths[0]);
-	}
-
-	@Test
-	public void getMapFor() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertEquals("0", metrics.get("abc.metric1"));
-	}
-
-	@Test
-	public void getMapForNull() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = fetcher.getMetricStore();
-
-		JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertNotNull(metrics);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java
deleted file mode 100644
index b26ceab..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java
+++ /dev/null
@@ -1,86 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.flink.runtime.webmonitor.metrics.JobMetricsHandler.PARAMETER_JOB_ID;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests for the JobMetricsHandler.
- */
-public class JobMetricsHandlerTest extends TestLogger {
-	@Test
-	public void testGetPaths() {
-		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/metrics", paths[0]);
-	}
-
-	@Test
-	public void getMapFor() throws Exception {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		pathParams.put(PARAMETER_JOB_ID, "jobid");
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertEquals("2", metrics.get("abc.metric3"));
-	}
-
-	@Test
-	public void getMapForNull() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = fetcher.getMetricStore();
-
-		JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertNull(metrics);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java
deleted file mode 100644
index d637a4a..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java
+++ /dev/null
@@ -1,90 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.flink.runtime.webmonitor.metrics.JobMetricsHandler.PARAMETER_JOB_ID;
-import static org.apache.flink.runtime.webmonitor.metrics.JobVertexMetricsHandler.PARAMETER_VERTEX_ID;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests for the JobVertexMetricsHandler.
- */
-public class JobVertexMetricsHandlerTest extends TestLogger {
-	@Test
-	public void testGetPaths() {
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/metrics", paths[0]);
-	}
-
-	@Test
-	public void getMapFor() throws Exception {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		pathParams.put(PARAMETER_JOB_ID, "jobid");
-		pathParams.put(PARAMETER_VERTEX_ID, "taskid");
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertEquals("3", metrics.get("8.abc.metric4"));
-
-		assertEquals("4", metrics.get("8.opname.abc.metric5"));
-	}
-
-	@Test
-	public void getMapForNull() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = fetcher.getMetricStore();
-
-		JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertNull(metrics);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcherTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcherTest.java
deleted file mode 100644
index 09a0829..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcherTest.java
+++ /dev/null
@@ -1,195 +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.webmonitor.metrics;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.Histogram;
-import org.apache.flink.metrics.Meter;
-import org.apache.flink.metrics.SimpleCounter;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.webmonitor.JobDetails;
-import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
-import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
-import org.apache.flink.runtime.metrics.dump.MetricQueryService;
-import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
-import org.apache.flink.runtime.metrics.util.TestingHistogram;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaJobManagerRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.eq;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-/**
- * Tests for the MetricFetcher.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(MetricFetcher.class)
-public class MetricFetcherTest extends TestLogger {
-	@Test
-	public void testUpdate() throws Exception {
-		final Time timeout = Time.seconds(10L);
-
-		// ========= setup TaskManager =================================================================================
-		JobID jobID = new JobID();
-		InstanceID tmID = new InstanceID();
-		ResourceID tmRID = new ResourceID(tmID.toString());
-		TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class);
-		when(taskManagerGateway.getAddress()).thenReturn("/tm/address");
-
-		Instance taskManager = mock(Instance.class);
-		when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway);
-		when(taskManager.getId()).thenReturn(tmID);
-		when(taskManager.getTaskManagerID()).thenReturn(tmRID);
-
-		// ========= setup JobManager ==================================================================================
-		JobDetails details = mock(JobDetails.class);
-		when(details.getJobId()).thenReturn(jobID);
-
-		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
-
-		when(jobManagerGateway.requestJobDetails(anyBoolean(), anyBoolean(), any(Time.class)))
-			.thenReturn(CompletableFuture.completedFuture(new MultipleJobsDetails(new JobDetails[0], new JobDetails[0])));
-		when(jobManagerGateway.requestTaskManagerInstances(any(Time.class)))
-			.thenReturn(CompletableFuture.completedFuture(Collections.singleton(taskManager)));
-		when(jobManagerGateway.getAddress()).thenReturn("/jm/address");
-
-		GatewayRetriever<JobManagerGateway> retriever = mock(AkkaJobManagerRetriever.class);
-		when(retriever.getNow())
-			.thenReturn(Optional.of(jobManagerGateway));
-
-		// ========= setup QueryServices ================================================================================
-		MetricQueryServiceGateway jmQueryService = mock(MetricQueryServiceGateway.class);
-		MetricQueryServiceGateway tmQueryService = mock(MetricQueryServiceGateway.class);
-
-		MetricDumpSerialization.MetricSerializationResult requestMetricsAnswer = createRequestDumpAnswer(tmID, jobID);
-
-		when(jmQueryService.queryMetrics(any(Time.class)))
-			.thenReturn(CompletableFuture.completedFuture(new MetricDumpSerialization.MetricSerializationResult(new byte[0], 0, 0, 0, 0)));
-		when(tmQueryService.queryMetrics(any(Time.class)))
-			.thenReturn(CompletableFuture.completedFuture(requestMetricsAnswer));
-
-		MetricQueryServiceRetriever queryServiceRetriever = mock(MetricQueryServiceRetriever.class);
-		when(queryServiceRetriever.retrieveService(eq("/jm/" + MetricQueryService.METRIC_QUERY_SERVICE_NAME))).thenReturn(CompletableFuture.completedFuture(jmQueryService));
-		when(queryServiceRetriever.retrieveService(eq("/tm/" + MetricQueryService.METRIC_QUERY_SERVICE_NAME + "_" + tmRID.getResourceIdString()))).thenReturn(CompletableFuture.completedFuture(tmQueryService));
-
-		// ========= start MetricFetcher testing =======================================================================
-		MetricFetcher fetcher = new MetricFetcher(
-			retriever,
-			queryServiceRetriever,
-			Executors.directExecutor(),
-			timeout);
-
-		// verify that update fetches metrics and updates the store
-		fetcher.update();
-		MetricStore store = fetcher.getMetricStore();
-		synchronized (store) {
-			assertEquals("7", store.jobManager.metrics.get("abc.hist_min"));
-			assertEquals("6", store.jobManager.metrics.get("abc.hist_max"));
-			assertEquals("4.0", store.jobManager.metrics.get("abc.hist_mean"));
-			assertEquals("0.5", store.jobManager.metrics.get("abc.hist_median"));
-			assertEquals("5.0", store.jobManager.metrics.get("abc.hist_stddev"));
-			assertEquals("0.75", store.jobManager.metrics.get("abc.hist_p75"));
-			assertEquals("0.9", store.jobManager.metrics.get("abc.hist_p90"));
-			assertEquals("0.95", store.jobManager.metrics.get("abc.hist_p95"));
-			assertEquals("0.98", store.jobManager.metrics.get("abc.hist_p98"));
-			assertEquals("0.99", store.jobManager.metrics.get("abc.hist_p99"));
-			assertEquals("0.999", store.jobManager.metrics.get("abc.hist_p999"));
-
-			assertEquals("x", store.getTaskManagerMetricStore(tmID.toString()).metrics.get("abc.gauge"));
-			assertEquals("5.0", store.getJobMetricStore(jobID.toString()).metrics.get("abc.jc"));
-			assertEquals("2", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.abc.tc"));
-			assertEquals("1", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.opname.abc.oc"));
-		}
-	}
-
-	private static MetricDumpSerialization.MetricSerializationResult createRequestDumpAnswer(InstanceID tmID, JobID jobID) {
-		Map<Counter, Tuple2<QueryScopeInfo, String>> counters = new HashMap<>();
-		Map<Gauge<?>, Tuple2<QueryScopeInfo, String>> gauges = new HashMap<>();
-		Map<Histogram, Tuple2<QueryScopeInfo, String>> histograms = new HashMap<>();
-		Map<Meter, Tuple2<QueryScopeInfo, String>> meters = new HashMap<>();
-
-		SimpleCounter c1 = new SimpleCounter();
-		SimpleCounter c2 = new SimpleCounter();
-
-		c1.inc(1);
-		c2.inc(2);
-
-		counters.put(c1, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.OperatorQueryScopeInfo(jobID.toString(), "taskid", 2, "opname", "abc"), "oc"));
-		counters.put(c2, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.TaskQueryScopeInfo(jobID.toString(), "taskid", 2, "abc"), "tc"));
-		meters.put(new Meter() {
-			@Override
-			public void markEvent() {
-			}
-
-			@Override
-			public void markEvent(long n) {
-			}
-
-			@Override
-			public double getRate() {
-				return 5;
-			}
-
-			@Override
-			public long getCount() {
-				return 10;
-			}
-		}, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.JobQueryScopeInfo(jobID.toString(), "abc"), "jc"));
-		gauges.put(new Gauge<String>() {
-			@Override
-			public String getValue() {
-				return "x";
-			}
-		}, new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmID.toString(), "abc"), "gauge"));
-		histograms.put(new TestingHistogram(), new Tuple2<QueryScopeInfo, String>(new QueryScopeInfo.JobManagerQueryScopeInfo("abc"), "hist"));
-
-		MetricDumpSerialization.MetricDumpSerializer serializer = new MetricDumpSerialization.MetricDumpSerializer();
-		MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, gauges, histograms, meters);
-		serializer.close();
-
-		return dump;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricStoreTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricStoreTest.java
deleted file mode 100644
index d19e8c6..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/MetricStoreTest.java
+++ /dev/null
@@ -1,88 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.metrics.dump.MetricDump;
-import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for the MetricStore.
- */
-public class MetricStoreTest extends TestLogger {
-	@Test
-	public void testAdd() throws IOException {
-		MetricStore store = setupStore(new MetricStore());
-
-		assertEquals("0", store.getJobManagerMetricStore().getMetric("abc.metric1", "-1"));
-		assertEquals("1", store.getTaskManagerMetricStore("tmid").getMetric("abc.metric2", "-1"));
-		assertEquals("2", store.getJobMetricStore("jobid").getMetric("abc.metric3", "-1"));
-		assertEquals("3", store.getTaskMetricStore("jobid", "taskid").getMetric("8.abc.metric4", "-1"));
-		assertEquals("4", store.getTaskMetricStore("jobid", "taskid").getMetric("8.opname.abc.metric5", "-1"));
-	}
-
-	@Test
-	public void testMalformedNameHandling() {
-		MetricStore store = new MetricStore();
-		//-----verify that no exceptions are thrown
-
-		// null
-		store.add(null);
-		// empty name
-		QueryScopeInfo.JobManagerQueryScopeInfo info = new QueryScopeInfo.JobManagerQueryScopeInfo("");
-		MetricDump.CounterDump cd = new MetricDump.CounterDump(info, "", 0);
-		store.add(cd);
-
-		//-----verify that no side effects occur
-		assertEquals(0, store.jobManager.metrics.size());
-		assertEquals(0, store.taskManagers.size());
-		assertEquals(0, store.jobs.size());
-	}
-
-	public static MetricStore setupStore(MetricStore store) {
-		QueryScopeInfo.JobManagerQueryScopeInfo jm = new QueryScopeInfo.JobManagerQueryScopeInfo("abc");
-		MetricDump.CounterDump cd1 = new MetricDump.CounterDump(jm, "metric1", 0);
-
-		QueryScopeInfo.TaskManagerQueryScopeInfo tm = new QueryScopeInfo.TaskManagerQueryScopeInfo("tmid", "abc");
-		MetricDump.CounterDump cd2 = new MetricDump.CounterDump(tm, "metric2", 1);
-
-		QueryScopeInfo.JobQueryScopeInfo job = new QueryScopeInfo.JobQueryScopeInfo("jobid", "abc");
-		MetricDump.CounterDump cd3 = new MetricDump.CounterDump(job, "metric3", 2);
-
-		QueryScopeInfo.TaskQueryScopeInfo task = new QueryScopeInfo.TaskQueryScopeInfo("jobid", "taskid", 8, "abc");
-		MetricDump.CounterDump cd4 = new MetricDump.CounterDump(task, "metric4", 3);
-
-		QueryScopeInfo.OperatorQueryScopeInfo operator = new QueryScopeInfo.OperatorQueryScopeInfo("jobid", "taskid", 8, "opname", "abc");
-		MetricDump.CounterDump cd5 = new MetricDump.CounterDump(operator, "metric5", 4);
-
-		store.add(cd1);
-		store.add(cd2);
-		store.add(cd3);
-		store.add(cd4);
-		store.add(cd5);
-
-		return store;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
deleted file mode 100644
index 9c5549e..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
+++ /dev/null
@@ -1,86 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler.TASK_MANAGER_ID_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests for the TaskManagerMetricsHandler.
- */
-public class TaskManagerMetricsHandlerTest extends TestLogger {
-	@Test
-	public void testGetPaths() {
-		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/taskmanagers/:taskmanagerid/metrics", paths[0]);
-	}
-
-	@Test
-	public void getMapFor() throws Exception {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore());
-
-		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-		pathParams.put(TASK_MANAGER_ID_KEY, "tmid");
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertEquals("1", metrics.get("abc.metric2"));
-	}
-
-	@Test
-	public void getMapForNull() {
-		MetricFetcher fetcher = new MetricFetcher(
-			mock(GatewayRetriever.class),
-			mock(MetricQueryServiceRetriever.class),
-			Executors.directExecutor(),
-			TestingUtils.TIMEOUT());
-		MetricStore store = fetcher.getMetricStore();
-
-		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher);
-
-		Map<String, String> pathParams = new HashMap<>();
-
-		Map<String, String> metrics = handler.getMapFor(pathParams, store);
-
-		assertNull(metrics);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionBuilder.java
deleted file mode 100644
index 979d943..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionBuilder.java
+++ /dev/null
@@ -1,150 +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.webmonitor.utils;
-
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.MeterView;
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.ArchivedExecution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.IOMetrics;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.util.Preconditions;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
-/**
- * Utility class for constructing an ArchivedExecution.
- */
-public class ArchivedExecutionBuilder {
-	private ExecutionAttemptID attemptId;
-	private long[] stateTimestamps;
-	private int attemptNumber;
-	private ExecutionState state;
-	private String failureCause;
-	private TaskManagerLocation assignedResourceLocation;
-	private StringifiedAccumulatorResult[] userAccumulators;
-	private IOMetrics ioMetrics;
-	private int parallelSubtaskIndex;
-
-	public ArchivedExecutionBuilder setAttemptId(ExecutionAttemptID attemptId) {
-		this.attemptId = attemptId;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setStateTimestamps(long[] stateTimestamps) {
-		Preconditions.checkArgument(stateTimestamps.length == ExecutionState.values().length);
-		this.stateTimestamps = stateTimestamps;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setAttemptNumber(int attemptNumber) {
-		this.attemptNumber = attemptNumber;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setState(ExecutionState state) {
-		this.state = state;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setFailureCause(String failureCause) {
-		this.failureCause = failureCause;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setAssignedResourceLocation(TaskManagerLocation assignedResourceLocation) {
-		this.assignedResourceLocation = assignedResourceLocation;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setUserAccumulators(StringifiedAccumulatorResult[] userAccumulators) {
-		this.userAccumulators = userAccumulators;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setParallelSubtaskIndex(int parallelSubtaskIndex) {
-		this.parallelSubtaskIndex = parallelSubtaskIndex;
-		return this;
-	}
-
-	public ArchivedExecutionBuilder setIOMetrics(IOMetrics ioMetrics) {
-		this.ioMetrics = ioMetrics;
-		return this;
-	}
-
-	public ArchivedExecution build() throws UnknownHostException {
-		return new ArchivedExecution(
-			userAccumulators != null ? userAccumulators : new StringifiedAccumulatorResult[0],
-			ioMetrics != null ? ioMetrics : new TestIOMetrics(),
-			attemptId != null ? attemptId : new ExecutionAttemptID(),
-			attemptNumber,
-			state != null ? state : ExecutionState.FINISHED,
-			failureCause != null ? failureCause : "(null)",
-			assignedResourceLocation != null ? assignedResourceLocation : new TaskManagerLocation(new ResourceID("tm"), InetAddress.getLocalHost(), 1234),
-			parallelSubtaskIndex,
-			stateTimestamps != null ? stateTimestamps : new long[]{1, 2, 3, 4, 5, 5, 5, 5}
-		);
-	}
-
-	private static class TestIOMetrics extends IOMetrics {
-		private static final long serialVersionUID = -5920076211680012555L;
-
-		public TestIOMetrics() {
-			super(
-				new MeterView(new TestCounter(1), 0),
-				new MeterView(new TestCounter(2), 0),
-				new MeterView(new TestCounter(3), 0),
-				new MeterView(new TestCounter(4), 0),
-				new MeterView(new TestCounter(5), 0));
-		}
-	}
-
-	private static class TestCounter implements Counter {
-		private final long count;
-
-		private TestCounter(long count) {
-			this.count = count;
-		}
-
-		@Override
-		public void inc() {
-		}
-
-		@Override
-		public void inc(long n) {
-		}
-
-		@Override
-		public void dec() {
-		}
-
-		@Override
-		public void dec(long n) {
-		}
-
-		@Override
-		public long getCount() {
-			return count;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionConfigBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionConfigBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionConfigBuilder.java
deleted file mode 100644
index 053f718..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionConfigBuilder.java
+++ /dev/null
@@ -1,71 +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.webmonitor.utils;
-
-import org.apache.flink.api.common.ArchivedExecutionConfig;
-import org.apache.flink.api.common.ExecutionMode;
-
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * Utility class for constructing an ArchivedExecutionConfig.
- */
-public class ArchivedExecutionConfigBuilder {
-	private String executionMode;
-	private String restartStrategyDescription;
-	private int parallelism;
-	private boolean objectReuseEnabled;
-	private Map<String, String> globalJobParameters;
-
-	public ArchivedExecutionConfigBuilder setExecutionMode(String executionMode) {
-		this.executionMode = executionMode;
-		return this;
-	}
-
-	public ArchivedExecutionConfigBuilder setRestartStrategyDescription(String restartStrategyDescription) {
-		this.restartStrategyDescription = restartStrategyDescription;
-		return this;
-	}
-
-	public ArchivedExecutionConfigBuilder setParallelism(int parallelism) {
-		this.parallelism = parallelism;
-		return this;
-	}
-
-	public ArchivedExecutionConfigBuilder setObjectReuseEnabled(boolean objectReuseEnabled) {
-		this.objectReuseEnabled = objectReuseEnabled;
-		return this;
-	}
-
-	public ArchivedExecutionConfigBuilder setGlobalJobParameters(Map<String, String> globalJobParameters) {
-		this.globalJobParameters = globalJobParameters;
-		return this;
-	}
-
-	public ArchivedExecutionConfig build() {
-		return new ArchivedExecutionConfig(
-			executionMode != null ? executionMode : ExecutionMode.PIPELINED.name(),
-			restartStrategyDescription != null ? restartStrategyDescription : "default",
-			parallelism,
-			objectReuseEnabled,
-			globalJobParameters != null ? globalJobParameters : Collections.<String, String>emptyMap()
-		);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
deleted file mode 100644
index 57b300a..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
+++ /dev/null
@@ -1,140 +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.webmonitor.utils;
-
-import org.apache.flink.api.common.ArchivedExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ErrorInfo;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.SerializedValue;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-/**
- * Utility class for constructing an ArchivedExecutionGraph.
- */
-public class ArchivedExecutionGraphBuilder {
-
-	private static final Random RANDOM = new Random();
-
-	private JobID jobID;
-	private String jobName;
-	private Map<JobVertexID, ArchivedExecutionJobVertex> tasks;
-	private List<ArchivedExecutionJobVertex> verticesInCreationOrder;
-	private long[] stateTimestamps;
-	private JobStatus state;
-	private ErrorInfo failureCause;
-	private String jsonPlan;
-	private StringifiedAccumulatorResult[] archivedUserAccumulators;
-	private ArchivedExecutionConfig archivedExecutionConfig;
-	private boolean isStoppable;
-	private Map<String, SerializedValue<Object>> serializedUserAccumulators;
-
-	public ArchivedExecutionGraphBuilder setJobID(JobID jobID) {
-		this.jobID = jobID;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setJobName(String jobName) {
-		this.jobName = jobName;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setTasks(Map<JobVertexID, ArchivedExecutionJobVertex> tasks) {
-		this.tasks = tasks;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setVerticesInCreationOrder(List<ArchivedExecutionJobVertex> verticesInCreationOrder) {
-		this.verticesInCreationOrder = verticesInCreationOrder;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setStateTimestamps(long[] stateTimestamps) {
-		Preconditions.checkArgument(stateTimestamps.length == JobStatus.values().length);
-		this.stateTimestamps = stateTimestamps;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setState(JobStatus state) {
-		this.state = state;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setFailureCause(ErrorInfo failureCause) {
-		this.failureCause = failureCause;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setJsonPlan(String jsonPlan) {
-		this.jsonPlan = jsonPlan;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setArchivedUserAccumulators(StringifiedAccumulatorResult[] archivedUserAccumulators) {
-		this.archivedUserAccumulators = archivedUserAccumulators;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setArchivedExecutionConfig(ArchivedExecutionConfig archivedExecutionConfig) {
-		this.archivedExecutionConfig = archivedExecutionConfig;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setStoppable(boolean stoppable) {
-		isStoppable = stoppable;
-		return this;
-	}
-
-	public ArchivedExecutionGraphBuilder setSerializedUserAccumulators(Map<String, SerializedValue<Object>> serializedUserAccumulators) {
-		this.serializedUserAccumulators = serializedUserAccumulators;
-		return this;
-	}
-
-	public ArchivedExecutionGraph build() {
-		Preconditions.checkNotNull(tasks, "Tasks must not be null.");
-		JobID jobID = this.jobID != null ? this.jobID : new JobID();
-		String jobName = this.jobName != null ? this.jobName : "job_" + RANDOM.nextInt();
-		return new ArchivedExecutionGraph(
-			jobID,
-			jobName,
-			tasks,
-			verticesInCreationOrder != null ? verticesInCreationOrder : new ArrayList<>(tasks.values()),
-			stateTimestamps != null ? stateTimestamps : new long[JobStatus.values().length],
-			state != null ? state : JobStatus.FINISHED,
-			failureCause,
-			jsonPlan != null ? jsonPlan : "{\"jobid\":\"" + jobID + "\", \"name\":\"" + jobName + "\", \"nodes\":[]}",
-			archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0],
-			serializedUserAccumulators != null ? serializedUserAccumulators : Collections.<String, SerializedValue<Object>>emptyMap(),
-			archivedExecutionConfig != null ? archivedExecutionConfig : new ArchivedExecutionConfigBuilder().build(),
-			isStoppable,
-			null,
-			null
-		);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionJobVertexBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionJobVertexBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionJobVertexBuilder.java
deleted file mode 100644
index 3ef4106..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionJobVertexBuilder.java
+++ /dev/null
@@ -1,84 +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.webmonitor.utils;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Random;
-
-/**
- * Utility class for constructing an ArchivedExecutionJobVertex.
- */
-public class ArchivedExecutionJobVertexBuilder {
-
-	private static final Random RANDOM = new Random();
-
-	private ArchivedExecutionVertex[] taskVertices;
-	private JobVertexID id;
-	private String name;
-	private int parallelism;
-	private int maxParallelism;
-	private StringifiedAccumulatorResult[] archivedUserAccumulators;
-
-	public ArchivedExecutionJobVertexBuilder setTaskVertices(ArchivedExecutionVertex[] taskVertices) {
-		this.taskVertices = taskVertices;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertexBuilder setId(JobVertexID id) {
-		this.id = id;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertexBuilder setName(String name) {
-		this.name = name;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertexBuilder setParallelism(int parallelism) {
-		this.parallelism = parallelism;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertexBuilder setMaxParallelism(int maxParallelism) {
-		this.maxParallelism = maxParallelism;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertexBuilder setArchivedUserAccumulators(StringifiedAccumulatorResult[] archivedUserAccumulators) {
-		this.archivedUserAccumulators = archivedUserAccumulators;
-		return this;
-	}
-
-	public ArchivedExecutionJobVertex build() {
-		Preconditions.checkNotNull(taskVertices);
-		return new ArchivedExecutionJobVertex(
-			taskVertices,
-			id != null ? id : new JobVertexID(),
-			name != null ? name : "task_" + RANDOM.nextInt(),
-			parallelism,
-			maxParallelism,
-			archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0]
-		);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionVertexBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionVertexBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionVertexBuilder.java
deleted file mode 100644
index 67e9e11..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionVertexBuilder.java
+++ /dev/null
@@ -1,73 +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.webmonitor.utils;
-
-import org.apache.flink.runtime.executiongraph.ArchivedExecution;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
-import org.apache.flink.runtime.util.EvictingBoundedList;
-import org.apache.flink.util.Preconditions;
-
-import java.util.List;
-import java.util.Random;
-
-/**
- * Utility class for constructing an ArchivedExecutionVertex.
- */
-public class ArchivedExecutionVertexBuilder {
-
-	private static final Random RANDOM = new Random();
-
-	private int subtaskIndex;
-	private EvictingBoundedList<ArchivedExecution> priorExecutions;
-	private String taskNameWithSubtask;
-	private ArchivedExecution currentExecution;
-
-	public ArchivedExecutionVertexBuilder setSubtaskIndex(int subtaskIndex) {
-		this.subtaskIndex = subtaskIndex;
-		return this;
-	}
-
-	public ArchivedExecutionVertexBuilder setPriorExecutions(List<ArchivedExecution> priorExecutions) {
-		this.priorExecutions = new EvictingBoundedList<>(priorExecutions.size());
-		for (ArchivedExecution execution : priorExecutions) {
-			this.priorExecutions.add(execution);
-		}
-		return this;
-	}
-
-	public ArchivedExecutionVertexBuilder setTaskNameWithSubtask(String taskNameWithSubtask) {
-		this.taskNameWithSubtask = taskNameWithSubtask;
-		return this;
-	}
-
-	public ArchivedExecutionVertexBuilder setCurrentExecution(ArchivedExecution currentExecution) {
-		this.currentExecution = currentExecution;
-		return this;
-	}
-
-	public ArchivedExecutionVertex build() {
-		Preconditions.checkNotNull(currentExecution);
-		return new ArchivedExecutionVertex(
-			subtaskIndex,
-			taskNameWithSubtask != null ? taskNameWithSubtask : "task_" + RANDOM.nextInt() + "_" + subtaskIndex,
-			currentExecution,
-			priorExecutions != null ? priorExecutions : new EvictingBoundedList<ArchivedExecution>(0)
-		);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
deleted file mode 100644
index 3e4fc01..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
+++ /dev/null
@@ -1,164 +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.webmonitor.utils;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ArchivedExecution;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ErrorInfo;
-import org.apache.flink.runtime.executiongraph.IOMetrics;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-
-import java.net.InetAddress;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Common entry-point for accessing generated ArchivedExecution* components.
- */
-public class ArchivedJobGenerationUtils {
-	public static final ObjectMapper MAPPER = new ObjectMapper();
-	public static final JsonFactory JACKSON_FACTORY = new JsonFactory()
-		.enable(JsonGenerator.Feature.AUTO_CLOSE_TARGET)
-		.disable(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT);
-
-	private static ArchivedExecutionGraph originalJob;
-	private static ArchivedExecutionJobVertex originalTask;
-	private static ArchivedExecutionVertex originalSubtask;
-	private static ArchivedExecution originalAttempt;
-
-	private static final Object lock = new Object();
-
-	private ArchivedJobGenerationUtils() {
-	}
-
-	public static AccessExecutionGraph getTestJob() throws Exception {
-		synchronized (lock) {
-			if (originalJob == null) {
-				generateArchivedJob();
-			}
-		}
-		return originalJob;
-	}
-
-	public static AccessExecutionJobVertex getTestTask() throws Exception {
-		synchronized (lock) {
-			if (originalJob == null) {
-				generateArchivedJob();
-			}
-		}
-		return originalTask;
-	}
-
-	public static AccessExecutionVertex getTestSubtask() throws Exception {
-		synchronized (lock) {
-			if (originalJob == null) {
-				generateArchivedJob();
-			}
-		}
-		return originalSubtask;
-	}
-
-	public static AccessExecution getTestAttempt() throws Exception {
-		synchronized (lock) {
-			if (originalJob == null) {
-				generateArchivedJob();
-			}
-		}
-		return originalAttempt;
-	}
-
-	private static void generateArchivedJob() throws Exception {
-		// Attempt
-		StringifiedAccumulatorResult acc1 = new StringifiedAccumulatorResult("name1", "type1", "value1");
-		StringifiedAccumulatorResult acc2 = new StringifiedAccumulatorResult("name2", "type2", "value2");
-		TaskManagerLocation location = new TaskManagerLocation(new ResourceID("hello"), InetAddress.getLocalHost(), 1234);
-		originalAttempt = new ArchivedExecutionBuilder()
-			.setStateTimestamps(new long[]{1, 2, 3, 4, 5, 6, 7, 8, 9})
-			.setParallelSubtaskIndex(1)
-			.setAttemptNumber(0)
-			.setAssignedResourceLocation(location)
-			.setUserAccumulators(new StringifiedAccumulatorResult[]{acc1, acc2})
-			.setState(ExecutionState.FINISHED)
-			.setFailureCause("attemptException")
-			.build();
-		// Subtask
-		originalSubtask = new ArchivedExecutionVertexBuilder()
-			.setSubtaskIndex(originalAttempt.getParallelSubtaskIndex())
-			.setTaskNameWithSubtask("hello(1/1)")
-			.setCurrentExecution(originalAttempt)
-			.build();
-		// Task
-		originalTask = new ArchivedExecutionJobVertexBuilder()
-			.setTaskVertices(new ArchivedExecutionVertex[]{originalSubtask})
-			.build();
-		// Job
-		Map<JobVertexID, ArchivedExecutionJobVertex> tasks = new HashMap<>();
-		tasks.put(originalTask.getJobVertexId(), originalTask);
-		originalJob = new ArchivedExecutionGraphBuilder()
-			.setJobID(new JobID())
-			.setTasks(tasks)
-			.setFailureCause(new ErrorInfo(new Exception("jobException"), originalAttempt.getStateTimestamp(ExecutionState.FAILED)))
-			.setState(JobStatus.FINISHED)
-			.setStateTimestamps(new long[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
-			.setArchivedUserAccumulators(new StringifiedAccumulatorResult[]{acc1, acc2})
-			.build();
-	}
-
-	// ========================================================================
-	// utility methods
-	// ========================================================================
-
-	public static void compareStringifiedAccumulators(StringifiedAccumulatorResult[] expectedAccs, ArrayNode writtenAccs) {
-		assertEquals(expectedAccs.length, writtenAccs.size());
-		for (int x = 0; x < expectedAccs.length; x++) {
-			JsonNode acc = writtenAccs.get(x);
-
-			assertEquals(expectedAccs[x].getName(), acc.get("name").asText());
-			assertEquals(expectedAccs[x].getType(), acc.get("type").asText());
-			assertEquals(expectedAccs[x].getValue(), acc.get("value").asText());
-		}
-	}
-
-	public static void compareIoMetrics(IOMetrics expectedMetrics, JsonNode writtenMetrics) {
-		assertEquals(expectedMetrics.getNumBytesInTotal(), writtenMetrics.get("read-bytes").asLong());
-		assertEquals(expectedMetrics.getNumBytesOut(), writtenMetrics.get("write-bytes").asLong());
-		assertEquals(expectedMetrics.getNumRecordsIn(), writtenMetrics.get("read-records").asLong());
-		assertEquals(expectedMetrics.getNumRecordsOut(), writtenMetrics.get("write-records").asLong());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/NotFoundException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/NotFoundException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/NotFoundException.java
new file mode 100644
index 0000000..4ad1759
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/NotFoundException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rest;
+
+/**
+ * A special exception that indicates that an element was not found and that the
+ * request should be answered with a {@code 404} return code.
+ */
+public class NotFoundException extends Exception {
+
+	private static final long serialVersionUID = -4036006746423754639L;
+
+	public NotFoundException(String message) {
+		super(message);
+	}
+}


[05/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
new file mode 100644
index 0000000..da115ee
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
@@ -0,0 +1,363 @@
+/*
+ * 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.rest.handler.legacy.files;
+
+/*****************************************************************************
+ * This code is based on the "HttpStaticFileServerHandler" from the
+ * Netty project's HTTP server example.
+ *
+ * See http://netty.io and
+ * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
+ *****************************************************************************/
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.rest.handler.RedirectHandler;
+import org.apache.flink.runtime.rest.handler.util.MimeTypes;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
+import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.Locale;
+import java.util.TimeZone;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.DATE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.EXPIRES;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.LAST_MODIFIED;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Simple file server handler that serves requests to web frontend's static files, such as
+ * HTML, CSS, or JS files.
+ *
+ * <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
+ * example.</p>
+ */
+@ChannelHandler.Sharable
+public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectHandler<T> {
+
+	/** Timezone in which this server answers its "if-modified" requests. */
+	private static final TimeZone GMT_TIMEZONE = TimeZone.getTimeZone("GMT");
+
+	/** Date format for HTTP. */
+	public static final String HTTP_DATE_FORMAT = "EEE, dd MMM yyyy HH:mm:ss zzz";
+
+	/** Be default, we allow files to be cached for 5 minutes. */
+	private static final int HTTP_CACHE_SECONDS = 300;
+
+	// ------------------------------------------------------------------------
+
+	/** The path in which the static documents are. */
+	private final File rootPath;
+
+	public StaticFileServerHandler(
+			GatewayRetriever<T> retriever,
+			CompletableFuture<String> localJobManagerAddressFuture,
+			Time timeout,
+			File rootPath) throws IOException {
+
+		super(localJobManagerAddressFuture, retriever, timeout);
+
+		this.rootPath = checkNotNull(rootPath).getCanonicalFile();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Responses to requests
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void respondAsLeader(ChannelHandlerContext channelHandlerContext, Routed routed, T gateway) throws Exception {
+		final HttpRequest request = routed.request();
+		final String requestPath;
+
+		// make sure we request the "index.html" in case there is a directory request
+		if (routed.path().endsWith("/")) {
+			requestPath = routed.path() + "index.html";
+		}
+		// in case the files being accessed are logs or stdout files, find appropriate paths.
+		else if (routed.path().equals("/jobmanager/log") || routed.path().equals("/jobmanager/stdout")) {
+			requestPath = "";
+		} else {
+			requestPath = routed.path();
+		}
+
+		respondToRequest(channelHandlerContext, request, requestPath);
+	}
+
+	/**
+	 * Response when running with leading JobManager.
+	 */
+	private void respondToRequest(ChannelHandlerContext ctx, HttpRequest request, String requestPath)
+			throws IOException, ParseException, URISyntaxException {
+
+		// convert to absolute path
+		final File file = new File(rootPath, requestPath);
+
+		if (!file.exists()) {
+			// file does not exist. Try to load it with the classloader
+			ClassLoader cl = StaticFileServerHandler.class.getClassLoader();
+
+			try (InputStream resourceStream = cl.getResourceAsStream("web" + requestPath)) {
+				boolean success = false;
+				try {
+					if (resourceStream != null) {
+						URL root = cl.getResource("web");
+						URL requested = cl.getResource("web" + requestPath);
+
+						if (root != null && requested != null) {
+							URI rootURI = new URI(root.getPath()).normalize();
+							URI requestedURI = new URI(requested.getPath()).normalize();
+
+							// Check that we don't load anything from outside of the
+							// expected scope.
+							if (!rootURI.relativize(requestedURI).equals(requestedURI)) {
+								logger.debug("Loading missing file from classloader: {}", requestPath);
+								// ensure that directory to file exists.
+								file.getParentFile().mkdirs();
+								Files.copy(resourceStream, file.toPath());
+
+								success = true;
+							}
+						}
+					}
+				} catch (Throwable t) {
+					logger.error("error while responding", t);
+				} finally {
+					if (!success) {
+						logger.debug("Unable to load requested file {} from classloader", requestPath);
+						sendError(ctx, NOT_FOUND);
+						return;
+					}
+				}
+			}
+		}
+
+		if (!file.exists() || file.isHidden() || file.isDirectory() || !file.isFile()) {
+			sendError(ctx, NOT_FOUND);
+			return;
+		}
+
+		if (!file.getCanonicalFile().toPath().startsWith(rootPath.toPath())) {
+			sendError(ctx, NOT_FOUND);
+			return;
+		}
+
+		// cache validation
+		final String ifModifiedSince = request.headers().get(IF_MODIFIED_SINCE);
+		if (ifModifiedSince != null && !ifModifiedSince.isEmpty()) {
+			SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
+			Date ifModifiedSinceDate = dateFormatter.parse(ifModifiedSince);
+
+			// Only compare up to the second because the datetime format we send to the client
+			// does not have milliseconds
+			long ifModifiedSinceDateSeconds = ifModifiedSinceDate.getTime() / 1000;
+			long fileLastModifiedSeconds = file.lastModified() / 1000;
+			if (ifModifiedSinceDateSeconds == fileLastModifiedSeconds) {
+				if (logger.isDebugEnabled()) {
+					logger.debug("Responding 'NOT MODIFIED' for file '" + file.getAbsolutePath() + '\'');
+				}
+
+				sendNotModified(ctx);
+				return;
+			}
+		}
+
+		if (logger.isDebugEnabled()) {
+			logger.debug("Responding with file '" + file.getAbsolutePath() + '\'');
+		}
+
+		// Don't need to close this manually. Netty's DefaultFileRegion will take care of it.
+		final RandomAccessFile raf;
+		try {
+			raf = new RandomAccessFile(file, "r");
+		}
+		catch (FileNotFoundException e) {
+			sendError(ctx, NOT_FOUND);
+			return;
+		}
+
+		try {
+			long fileLength = raf.length();
+
+			HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+			setContentTypeHeader(response, file);
+
+			// since the log and out files are rapidly changing, we don't want to browser to cache them
+			if (!(requestPath.contains("log") || requestPath.contains("out"))) {
+				setDateAndCacheHeaders(response, file);
+			}
+			if (HttpHeaders.isKeepAlive(request)) {
+				response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+			}
+			HttpHeaders.setContentLength(response, fileLength);
+
+			// write the initial line and the header.
+			ctx.write(response);
+
+			// write the content.
+			ChannelFuture lastContentFuture;
+			if (ctx.pipeline().get(SslHandler.class) == null) {
+				ctx.write(new DefaultFileRegion(raf.getChannel(), 0, fileLength), ctx.newProgressivePromise());
+				lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+			} else {
+				lastContentFuture = ctx.writeAndFlush(new HttpChunkedInput(new ChunkedFile(raf, 0, fileLength, 8192)),
+					ctx.newProgressivePromise());
+				// HttpChunkedInput will write the end marker (LastHttpContent) for us.
+			}
+
+			// close the connection, if no keep-alive is needed
+			if (!HttpHeaders.isKeepAlive(request)) {
+				lastContentFuture.addListener(ChannelFutureListener.CLOSE);
+			}
+		} catch (Exception e) {
+			raf.close();
+			logger.error("Failed to serve file.", e);
+			sendError(ctx, INTERNAL_SERVER_ERROR);
+		}
+	}
+
+	@Override
+	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+		if (ctx.channel().isActive()) {
+			logger.error("Caught exception", cause);
+			sendError(ctx, INTERNAL_SERVER_ERROR);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities to encode headers and responses
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Writes a simple  error response message.
+	 *
+	 * @param ctx    The channel context to write the response to.
+	 * @param status The response status.
+	 */
+	public static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
+		FullHttpResponse response = new DefaultFullHttpResponse(
+				HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8));
+		response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8");
+
+		// close the connection as soon as the error message is sent.
+		ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
+	}
+
+	/**
+	 * Send the "304 Not Modified" response. This response can be used when the
+	 * file timestamp is the same as what the browser is sending up.
+	 *
+	 * @param ctx The channel context to write the response to.
+	 */
+	public static void sendNotModified(ChannelHandlerContext ctx) {
+		FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, NOT_MODIFIED);
+		setDateHeader(response);
+
+		// close the connection as soon as the error message is sent.
+		ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
+	}
+
+	/**
+	 * Sets the "date" header for the HTTP response.
+	 *
+	 * @param response HTTP response
+	 */
+	public static void setDateHeader(FullHttpResponse response) {
+		SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
+		dateFormatter.setTimeZone(GMT_TIMEZONE);
+
+		Calendar time = new GregorianCalendar();
+		response.headers().set(DATE, dateFormatter.format(time.getTime()));
+	}
+
+	/**
+	 * Sets the "date" and "cache" headers for the HTTP Response.
+	 *
+	 * @param response    The HTTP response object.
+	 * @param fileToCache File to extract the modification timestamp from.
+	 */
+	public static void setDateAndCacheHeaders(HttpResponse response, File fileToCache) {
+		SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
+		dateFormatter.setTimeZone(GMT_TIMEZONE);
+
+		// date header
+		Calendar time = new GregorianCalendar();
+		response.headers().set(DATE, dateFormatter.format(time.getTime()));
+
+		// cache headers
+		time.add(Calendar.SECOND, HTTP_CACHE_SECONDS);
+		response.headers().set(EXPIRES, dateFormatter.format(time.getTime()));
+		response.headers().set(CACHE_CONTROL, "private, max-age=" + HTTP_CACHE_SECONDS);
+		response.headers().set(LAST_MODIFIED, dateFormatter.format(new Date(fileToCache.lastModified())));
+	}
+
+	/**
+	 * Sets the content type header for the HTTP Response.
+	 *
+	 * @param response HTTP response
+	 * @param file     file to extract content type
+	 */
+	public static void setContentTypeHeader(HttpResponse response, File file) {
+		String mimeType = MimeTypes.getMimeTypeForFileName(file.getName());
+		String mimeFinal = mimeType != null ? mimeType : MimeTypes.getDefaultMimeType();
+		response.headers().set(CONTENT_TYPE, mimeFinal);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandler.java
new file mode 100644
index 0000000..315bdc2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/AbstractMetricsHandler.java
@@ -0,0 +1,139 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
+import org.apache.flink.util.Preconditions;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Abstract request handler that returns a list of all available metrics or the values for a set of metrics.
+ *
+ * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
+ * {@code [ { "id" : "X" } ] }
+ *
+ * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
+ * {@code /get?X,Y}
+ * The handler will then return a list containing the values of the requested metrics.
+ * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
+ */
+public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler {
+	private final MetricFetcher fetcher;
+
+	public AbstractMetricsHandler(Executor executor, MetricFetcher fetcher) {
+		super(executor);
+		this.fetcher = Preconditions.checkNotNull(fetcher);
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				fetcher.update();
+				String requestedMetricsList = queryParams.get("get");
+				try {
+					return requestedMetricsList != null
+						? getMetricsValues(pathParams, requestedMetricsList)
+						: getAvailableMetricsList(pathParams);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not retrieve metrics.", e);
+				}
+			},
+			executor);
+
+	}
+
+	/**
+	 * Returns a Map containing the metrics belonging to the entity pointed to by the path parameters.
+	 *
+	 * @param pathParams REST path parameters
+	 * @param metrics MetricStore containing all metrics
+	 * @return Map containing metrics, or null if no metric exists
+	 */
+	protected abstract Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics);
+
+	private String getMetricsValues(Map<String, String> pathParams, String requestedMetricsList) throws IOException {
+		if (requestedMetricsList.isEmpty()) {
+			/*
+			 * The WebInterface doesn't check whether the list of available metrics was empty. This can lead to a
+			 * request for which the "get" parameter is an empty string.
+			 */
+			return "";
+		}
+		MetricStore metricStore = fetcher.getMetricStore();
+		synchronized (metricStore) {
+			Map<String, String> metrics = getMapFor(pathParams, metricStore);
+			if (metrics == null) {
+				return "";
+			}
+			String[] requestedMetrics = requestedMetricsList.split(",");
+
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+			gen.writeStartArray();
+			for (String requestedMetric : requestedMetrics) {
+				Object metricValue = metrics.get(requestedMetric);
+				if (metricValue != null) {
+					gen.writeStartObject();
+					gen.writeStringField("id", requestedMetric);
+					gen.writeStringField("value", metricValue.toString());
+					gen.writeEndObject();
+				}
+			}
+			gen.writeEndArray();
+
+			gen.close();
+			return writer.toString();
+		}
+	}
+
+	private String getAvailableMetricsList(Map<String, String> pathParams) throws IOException {
+		MetricStore metricStore = fetcher.getMetricStore();
+		synchronized (metricStore) {
+			Map<String, String> metrics = getMapFor(pathParams, metricStore);
+			if (metrics == null) {
+				return "";
+			}
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+			gen.writeStartArray();
+			for (String m : metrics.keySet()) {
+				gen.writeStartObject();
+				gen.writeStringField("id", m);
+				gen.writeEndObject();
+			}
+			gen.writeEndArray();
+
+			gen.close();
+			return writer.toString();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandler.java
new file mode 100644
index 0000000..c568ee0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobManagerMetricsHandler.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns for the job manager a list of all available metrics or the values for a set of metrics.
+ *
+ * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
+ * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
+ *
+ * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
+ * {@code /get?X,Y}
+ * The handler will then return a list containing the values of the requested metrics.
+ * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
+ */
+public class JobManagerMetricsHandler extends AbstractMetricsHandler {
+
+	private static final String JOBMANAGER_METRICS_REST_PATH = "/jobmanager/metrics";
+
+	public JobManagerMetricsHandler(Executor executor, MetricFetcher fetcher) {
+		super(executor, fetcher);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOBMANAGER_METRICS_REST_PATH};
+	}
+
+	@Override
+	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
+		MetricStore.JobManagerMetricStore jobManager = metrics.getJobManagerMetricStore();
+		if (jobManager == null) {
+			return null;
+		} else {
+			return jobManager.metrics;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandler.java
new file mode 100644
index 0000000..7341eb8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobMetricsHandler.java
@@ -0,0 +1,55 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns for a given job a list of all available metrics or the values for a set of metrics.
+ *
+ * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
+ * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
+ *
+ * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
+ * {@code /get?X,Y}
+ * The handler will then return a list containing the values of the requested metrics.
+ * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
+ */
+public class JobMetricsHandler extends AbstractMetricsHandler {
+	public static final String PARAMETER_JOB_ID = "jobid";
+	private static final String JOB_METRICS_REST_PATH = "/jobs/:jobid/metrics";
+
+	public JobMetricsHandler(Executor executor, MetricFetcher fetcher) {
+		super(executor, fetcher);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_METRICS_REST_PATH};
+	}
+
+	@Override
+	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
+		MetricStore.JobMetricStore job = metrics.getJobMetricStore(pathParams.get(PARAMETER_JOB_ID));
+		return job != null
+			? job.metrics
+			: null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandler.java
new file mode 100644
index 0000000..3a701ab
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/JobVertexMetricsHandler.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns for a given task a list of all available metrics or the values for a set of metrics.
+ *
+ * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
+ * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
+ *
+ * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
+ * {@code /get?X,Y}
+ * The handler will then return a list containing the values of the requested metrics.
+ * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
+ */
+public class JobVertexMetricsHandler extends AbstractMetricsHandler {
+	public static final String PARAMETER_VERTEX_ID = "vertexid";
+	private static final String JOB_VERTEX_METRICS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/metrics";
+
+	public JobVertexMetricsHandler(Executor executor, MetricFetcher fetcher) {
+		super(executor, fetcher);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_VERTEX_METRICS_REST_PATH};
+	}
+
+	@Override
+	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
+		MetricStore.TaskMetricStore task = metrics.getTaskMetricStore(
+			pathParams.get(JobMetricsHandler.PARAMETER_JOB_ID),
+			pathParams.get(PARAMETER_VERTEX_ID));
+		return task != null
+			? task.metrics
+			: null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcher.java
new file mode 100644
index 0000000..9f53808
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcher.java
@@ -0,0 +1,211 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
+import org.apache.flink.runtime.metrics.dump.MetricDump;
+import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
+import org.apache.flink.runtime.metrics.dump.MetricQueryService;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.MetricDumpDeserializer;
+
+/**
+ * The MetricFetcher can be used to fetch metrics from the JobManager and all registered TaskManagers.
+ *
+ * <p>Metrics will only be fetched when {@link MetricFetcher#update()} is called, provided that a sufficient time since
+ * the last call has passed.
+ */
+public class MetricFetcher {
+	private static final Logger LOG = LoggerFactory.getLogger(MetricFetcher.class);
+
+	private final GatewayRetriever<JobManagerGateway> retriever;
+	private final MetricQueryServiceRetriever queryServiceRetriever;
+	private final Executor executor;
+	private final Time timeout;
+
+	private final MetricStore metrics = new MetricStore();
+	private final MetricDumpDeserializer deserializer = new MetricDumpDeserializer();
+
+	private long lastUpdateTime;
+
+	public MetricFetcher(
+			GatewayRetriever<JobManagerGateway> retriever,
+			MetricQueryServiceRetriever queryServiceRetriever,
+			Executor executor,
+			Time timeout) {
+		this.retriever = Preconditions.checkNotNull(retriever);
+		this.queryServiceRetriever = Preconditions.checkNotNull(queryServiceRetriever);
+		this.executor = Preconditions.checkNotNull(executor);
+		this.timeout = Preconditions.checkNotNull(timeout);
+	}
+
+	/**
+	 * Returns the MetricStore containing all stored metrics.
+	 *
+	 * @return MetricStore containing all stored metrics;
+	 */
+	public MetricStore getMetricStore() {
+		return metrics;
+	}
+
+	/**
+	 * This method can be used to signal this MetricFetcher that the metrics are still in use and should be updated.
+	 */
+	public void update() {
+		synchronized (this) {
+			long currentTime = System.currentTimeMillis();
+			if (currentTime - lastUpdateTime > 10000) { // 10 seconds have passed since the last update
+				lastUpdateTime = currentTime;
+				fetchMetrics();
+			}
+		}
+	}
+
+	private void fetchMetrics() {
+		try {
+			Optional<JobManagerGateway> optJobManagerGateway = retriever.getNow();
+			if (optJobManagerGateway.isPresent()) {
+				final JobManagerGateway jobManagerGateway = optJobManagerGateway.get();
+
+				/**
+				 * Remove all metrics that belong to a job that is not running and no longer archived.
+				 */
+				CompletableFuture<MultipleJobsDetails> jobDetailsFuture = jobManagerGateway.requestJobDetails(true, true, timeout);
+
+				jobDetailsFuture.whenCompleteAsync(
+					(MultipleJobsDetails jobDetails, Throwable throwable) -> {
+						if (throwable != null) {
+							LOG.debug("Fetching of JobDetails failed.", throwable);
+						} else {
+							ArrayList<String> toRetain = new ArrayList<>();
+							for (JobDetails job : jobDetails.getRunningJobs()) {
+								toRetain.add(job.getJobId().toString());
+							}
+							for (JobDetails job : jobDetails.getFinishedJobs()) {
+								toRetain.add(job.getJobId().toString());
+							}
+							synchronized (metrics) {
+								metrics.jobs.keySet().retainAll(toRetain);
+							}
+						}
+					},
+					executor);
+
+				String jobManagerPath = jobManagerGateway.getAddress();
+				String jmQueryServicePath = jobManagerPath.substring(0, jobManagerPath.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME;
+
+				retrieveAndQueryMetrics(jmQueryServicePath);
+
+				/**
+				 * We first request the list of all registered task managers from the job manager, and then
+				 * request the respective metric dump from each task manager.
+				 *
+				 * <p>All stored metrics that do not belong to a registered task manager will be removed.
+				 */
+				CompletableFuture<Collection<Instance>> taskManagersFuture = jobManagerGateway.requestTaskManagerInstances(timeout);
+
+				taskManagersFuture.whenCompleteAsync(
+					(Collection<Instance> taskManagers, Throwable throwable) -> {
+						if (throwable != null) {
+							LOG.debug("Fetching list of registered TaskManagers failed.", throwable);
+						} else {
+							List<String> activeTaskManagers = taskManagers.stream().map(
+								taskManagerInstance -> {
+									final String taskManagerAddress = taskManagerInstance.getTaskManagerGateway().getAddress();
+									final String tmQueryServicePath = taskManagerAddress.substring(0, taskManagerAddress.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME + "_" + taskManagerInstance.getTaskManagerID().getResourceIdString();
+
+									retrieveAndQueryMetrics(tmQueryServicePath);
+
+									return taskManagerInstance.getId().toString();
+								}).collect(Collectors.toList());
+
+							synchronized (metrics) {
+								metrics.taskManagers.keySet().retainAll(activeTaskManagers);
+							}
+						}
+					},
+					executor);
+			}
+		} catch (Exception e) {
+			LOG.warn("Exception while fetching metrics.", e);
+		}
+	}
+
+	/**
+	 * Retrieves and queries the specified QueryServiceGateway.
+	 *
+	 * @param queryServicePath specifying the QueryServiceGateway
+	 */
+	private void retrieveAndQueryMetrics(String queryServicePath) {
+		final CompletableFuture<MetricQueryServiceGateway> queryServiceGatewayFuture = queryServiceRetriever.retrieveService(queryServicePath);
+
+		queryServiceGatewayFuture.whenCompleteAsync(
+			(MetricQueryServiceGateway queryServiceGateway, Throwable t) -> {
+				if (t != null) {
+					LOG.debug("Could not retrieve QueryServiceGateway.", t);
+				} else {
+					queryMetrics(queryServiceGateway);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Query the metrics from the given QueryServiceGateway.
+	 *
+	 * @param queryServiceGateway to query for metrics
+	 */
+	private void queryMetrics(final MetricQueryServiceGateway queryServiceGateway) {
+		queryServiceGateway
+			.queryMetrics(timeout)
+			.whenCompleteAsync(
+				(MetricDumpSerialization.MetricSerializationResult result, Throwable t) -> {
+					if (t != null) {
+						LOG.debug("Fetching metrics failed.", t);
+					} else {
+						List<MetricDump> dumpedMetrics = deserializer.deserialize(result);
+						synchronized (metrics) {
+							for (MetricDump metric : dumpedMetrics) {
+								metrics.add(metric);
+							}
+						}
+					}
+				},
+				executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStore.java
new file mode 100644
index 0000000..6d3fc99
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricStore.java
@@ -0,0 +1,305 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.metrics.dump.MetricDump;
+import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_COUNTER;
+import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_GAUGE;
+import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_HISTOGRAM;
+import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_METER;
+import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_JM;
+import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_JOB;
+import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_OPERATOR;
+import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_TASK;
+import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_TM;
+
+/**
+ * Nested data-structure to store metrics.
+ *
+ * <p>This structure is not thread-safe.
+ */
+public class MetricStore {
+	private static final Logger LOG = LoggerFactory.getLogger(MetricStore.class);
+
+	final JobManagerMetricStore jobManager = new JobManagerMetricStore();
+	final Map<String, TaskManagerMetricStore> taskManagers = new HashMap<>();
+	final Map<String, JobMetricStore> jobs = new HashMap<>();
+
+	// -----------------------------------------------------------------------------------------------------------------
+	// Adding metrics
+	// -----------------------------------------------------------------------------------------------------------------
+	public void add(MetricDump metric) {
+		try {
+			QueryScopeInfo info = metric.scopeInfo;
+			TaskManagerMetricStore tm;
+			JobMetricStore job;
+			TaskMetricStore task;
+			SubtaskMetricStore subtask;
+
+			String name = info.scope.isEmpty()
+				? metric.name
+				: info.scope + "." + metric.name;
+
+			if (name.isEmpty()) { // malformed transmission
+				return;
+			}
+
+			switch (info.getCategory()) {
+				case INFO_CATEGORY_JM:
+					addMetric(jobManager.metrics, name, metric);
+					break;
+				case INFO_CATEGORY_TM:
+					String tmID = ((QueryScopeInfo.TaskManagerQueryScopeInfo) info).taskManagerID;
+					tm = taskManagers.get(tmID);
+					if (tm == null) {
+						tm = new TaskManagerMetricStore();
+						taskManagers.put(tmID, tm);
+					}
+					if (name.contains("GarbageCollector")) {
+						String gcName = name.substring("Status.JVM.GarbageCollector.".length(), name.lastIndexOf('.'));
+						tm.addGarbageCollectorName(gcName);
+					}
+					addMetric(tm.metrics, name, metric);
+					break;
+				case INFO_CATEGORY_JOB:
+					QueryScopeInfo.JobQueryScopeInfo jobInfo = (QueryScopeInfo.JobQueryScopeInfo) info;
+					job = jobs.get(jobInfo.jobID);
+					if (job == null) {
+						job = new JobMetricStore();
+						jobs.put(jobInfo.jobID, job);
+					}
+					addMetric(job.metrics, name, metric);
+					break;
+				case INFO_CATEGORY_TASK:
+					QueryScopeInfo.TaskQueryScopeInfo taskInfo = (QueryScopeInfo.TaskQueryScopeInfo) info;
+					job = jobs.get(taskInfo.jobID);
+					if (job == null) {
+						job = new JobMetricStore();
+						jobs.put(taskInfo.jobID, job);
+					}
+					task = job.tasks.get(taskInfo.vertexID);
+					if (task == null) {
+						task = new TaskMetricStore();
+						job.tasks.put(taskInfo.vertexID, task);
+					}
+					subtask = task.subtasks.get(taskInfo.subtaskIndex);
+					if (subtask == null) {
+						subtask = new SubtaskMetricStore();
+						task.subtasks.put(taskInfo.subtaskIndex, subtask);
+					}
+					/**
+					 * The duplication is intended. Metrics scoped by subtask are useful for several job/task handlers,
+					 * while the WebInterface task metric queries currently do not account for subtasks, so we don't
+					 * divide by subtask and instead use the concatenation of subtask index and metric name as the name
+					 * for those.
+					 */
+					addMetric(subtask.metrics, name, metric);
+					addMetric(task.metrics, taskInfo.subtaskIndex + "." + name, metric);
+					break;
+				case INFO_CATEGORY_OPERATOR:
+					QueryScopeInfo.OperatorQueryScopeInfo operatorInfo = (QueryScopeInfo.OperatorQueryScopeInfo) info;
+					job = jobs.get(operatorInfo.jobID);
+					if (job == null) {
+						job = new JobMetricStore();
+						jobs.put(operatorInfo.jobID, job);
+					}
+					task = job.tasks.get(operatorInfo.vertexID);
+					if (task == null) {
+						task = new TaskMetricStore();
+						job.tasks.put(operatorInfo.vertexID, task);
+					}
+					/**
+					 * As the WebInterface does not account for operators (because it can't) we don't
+					 * divide by operator and instead use the concatenation of subtask index, operator name and metric name
+					 * as the name.
+					 */
+					addMetric(task.metrics, operatorInfo.subtaskIndex + "." + operatorInfo.operatorName + "." + name, metric);
+					break;
+				default:
+					LOG.debug("Invalid metric dump category: " + info.getCategory());
+			}
+		} catch (Exception e) {
+			LOG.debug("Malformed metric dump.", e);
+		}
+	}
+
+	private void addMetric(Map<String, String> target, String name, MetricDump metric) {
+		switch (metric.getCategory()) {
+			case METRIC_CATEGORY_COUNTER:
+				MetricDump.CounterDump counter = (MetricDump.CounterDump) metric;
+				target.put(name, String.valueOf(counter.count));
+				break;
+			case METRIC_CATEGORY_GAUGE:
+				MetricDump.GaugeDump gauge = (MetricDump.GaugeDump) metric;
+				target.put(name, gauge.value);
+				break;
+			case METRIC_CATEGORY_HISTOGRAM:
+				MetricDump.HistogramDump histogram = (MetricDump.HistogramDump) metric;
+				target.put(name + "_min", String.valueOf(histogram.min));
+				target.put(name + "_max", String.valueOf(histogram.max));
+				target.put(name + "_mean", String.valueOf(histogram.mean));
+				target.put(name + "_median", String.valueOf(histogram.median));
+				target.put(name + "_stddev", String.valueOf(histogram.stddev));
+				target.put(name + "_p75", String.valueOf(histogram.p75));
+				target.put(name + "_p90", String.valueOf(histogram.p90));
+				target.put(name + "_p95", String.valueOf(histogram.p95));
+				target.put(name + "_p98", String.valueOf(histogram.p98));
+				target.put(name + "_p99", String.valueOf(histogram.p99));
+				target.put(name + "_p999", String.valueOf(histogram.p999));
+				break;
+			case METRIC_CATEGORY_METER:
+				MetricDump.MeterDump meter = (MetricDump.MeterDump) metric;
+				target.put(name, String.valueOf(meter.rate));
+				break;
+		}
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	// Accessors for sub MetricStores
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Returns the {@link JobManagerMetricStore}.
+	 *
+	 * @return JobManagerMetricStore
+	 */
+	public JobManagerMetricStore getJobManagerMetricStore() {
+		return jobManager;
+	}
+
+	/**
+	 * Returns the {@link TaskManagerMetricStore} for the given taskmanager ID.
+	 *
+	 * @param tmID taskmanager ID
+	 * @return TaskManagerMetricStore for the given ID, or null if no store for the given argument exists
+	 */
+	public TaskManagerMetricStore getTaskManagerMetricStore(String tmID) {
+		return taskManagers.get(tmID);
+	}
+
+	/**
+	 * Returns the {@link JobMetricStore} for the given job ID.
+	 *
+	 * @param jobID job ID
+	 * @return JobMetricStore for the given ID, or null if no store for the given argument exists
+	 */
+	public JobMetricStore getJobMetricStore(String jobID) {
+		return jobs.get(jobID);
+	}
+
+	/**
+	 * Returns the {@link TaskMetricStore} for the given job/task ID.
+	 *
+	 * @param jobID  job ID
+	 * @param taskID task ID
+	 * @return TaskMetricStore for given IDs, or null if no store for the given arguments exists
+	 */
+	public TaskMetricStore getTaskMetricStore(String jobID, String taskID) {
+		JobMetricStore job = getJobMetricStore(jobID);
+		if (job == null) {
+			return null;
+		}
+		return job.getTaskMetricStore(taskID);
+	}
+
+	/**
+	 * Returns the {@link SubtaskMetricStore} for the given job/task ID and subtask index.
+	 *
+	 * @param jobID        job ID
+	 * @param taskID       task ID
+	 * @param subtaskIndex subtask index
+	 * @return SubtaskMetricStore for the given IDs and index, or null if no store for the given arguments exists
+	 */
+	public SubtaskMetricStore getSubtaskMetricStore(String jobID, String taskID, int subtaskIndex) {
+		TaskMetricStore task = getTaskMetricStore(jobID, taskID);
+		if (task == null) {
+			return null;
+		}
+		return task.getSubtaskMetricStore(subtaskIndex);
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	// sub MetricStore classes
+	// -----------------------------------------------------------------------------------------------------------------
+	private abstract static class ComponentMetricStore {
+		public final Map<String, String> metrics = new HashMap<>();
+
+		public String getMetric(String name, String defaultValue) {
+			String value = this.metrics.get(name);
+			return value != null
+				? value
+				: defaultValue;
+		}
+	}
+
+	/**
+	 * Sub-structure containing metrics of the JobManager.
+	 */
+	public static class JobManagerMetricStore extends ComponentMetricStore {
+	}
+
+	/**
+	 * Sub-structure containing metrics of a single TaskManager.
+	 */
+	public static class TaskManagerMetricStore extends ComponentMetricStore {
+		public final Set<String> garbageCollectorNames = new HashSet<>();
+
+		public void addGarbageCollectorName(String name) {
+			garbageCollectorNames.add(name);
+		}
+	}
+
+	/**
+	 * Sub-structure containing metrics of a single Job.
+	 */
+	public static class JobMetricStore extends ComponentMetricStore {
+		private final Map<String, TaskMetricStore> tasks = new HashMap<>();
+
+		public TaskMetricStore getTaskMetricStore(String taskID) {
+			return tasks.get(taskID);
+		}
+	}
+
+	/**
+	 * Sub-structure containing metrics of a single Task.
+	 */
+	public static class TaskMetricStore extends ComponentMetricStore {
+		private final Map<Integer, SubtaskMetricStore> subtasks = new HashMap<>();
+
+		public SubtaskMetricStore getSubtaskMetricStore(int subtaskIndex) {
+			return subtasks.get(subtaskIndex);
+		}
+	}
+
+	/**
+	 * Sub-structure containing metrics of a single Subtask.
+	 */
+	public static class SubtaskMetricStore extends ComponentMetricStore {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandler.java
new file mode 100644
index 0000000..90bafb7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/metrics/TaskManagerMetricsHandler.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rest.handler.legacy.metrics;
+
+import org.apache.flink.runtime.rest.handler.legacy.TaskManagersHandler;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns for a given task manager a list of all available metrics or the values for a set of metrics.
+ *
+ * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
+ * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
+ *
+ * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
+ * {@code /get?X,Y}
+ * The handler will then return a list containing the values of the requested metrics.
+ * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
+ */
+public class TaskManagerMetricsHandler extends AbstractMetricsHandler {
+
+	private static final String TASKMANAGER_METRICS_REST_PATH = "/taskmanagers/:taskmanagerid/metrics";
+
+	public TaskManagerMetricsHandler(Executor executor, MetricFetcher fetcher) {
+		super(executor, fetcher);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{TASKMANAGER_METRICS_REST_PATH};
+	}
+
+	@Override
+	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
+		MetricStore.TaskManagerMetricStore taskManager = metrics.getTaskManagerMetricStore(pathParams.get(TaskManagersHandler.TASK_MANAGER_ID_KEY));
+		if (taskManager == null) {
+			return null;
+		} else {
+			return taskManager.metrics;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/MutableIOMetrics.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/MutableIOMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/MutableIOMetrics.java
new file mode 100644
index 0000000..e2aaaf7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/MutableIOMetrics.java
@@ -0,0 +1,109 @@
+/*
+ * 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.rest.handler.util;
+
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.rest.handler.legacy.JobVertexDetailsHandler;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricStore;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/**
+ * This class is a mutable version of the {@link IOMetrics} class that allows adding up IO-related metrics.
+ *
+ * <p>For finished jobs these metrics are stored in the {@link ExecutionGraph} as another {@link IOMetrics}.
+ * For running jobs these metrics are retrieved using the {@link MetricFetcher}.
+ *
+ * <p>This class provides a common interface to handle both cases, reducing complexity in various handlers (like
+ * the {@link JobVertexDetailsHandler}).
+ */
+public class MutableIOMetrics extends IOMetrics {
+
+	private static final long serialVersionUID = -5460777634971381737L;
+
+	public MutableIOMetrics() {
+		super(0, 0, 0, 0, 0, 0.0D, 0.0D, 0.0D, 0.0D, 0.0D);
+	}
+
+	/**
+	 * Adds the IO metrics for the given attempt to this object. If the {@link AccessExecution} is in
+	 * a terminal state the contained {@link IOMetrics} object is added. Otherwise the given {@link MetricFetcher} is
+	 * used to retrieve the required metrics.
+	 *
+	 * @param attempt Attempt whose IO metrics should be added
+	 * @param fetcher MetricFetcher to retrieve metrics for running jobs
+	 * @param jobID JobID to which the attempt belongs
+	 * @param taskID TaskID to which the attempt belongs
+	 */
+	public void addIOMetrics(AccessExecution attempt, @Nullable MetricFetcher fetcher, String jobID, String taskID) {
+		if (attempt.getState().isTerminal()) {
+			IOMetrics ioMetrics = attempt.getIOMetrics();
+			if (ioMetrics != null) { // execAttempt is already finished, use final metrics stored in ExecutionGraph
+				this.numBytesInLocal += ioMetrics.getNumBytesInLocal();
+				this.numBytesInRemote += ioMetrics.getNumBytesInRemote();
+				this.numBytesOut += ioMetrics.getNumBytesOut();
+				this.numRecordsIn += ioMetrics.getNumRecordsIn();
+				this.numRecordsOut += ioMetrics.getNumRecordsOut();
+			}
+		} else { // execAttempt is still running, use MetricQueryService instead
+			if (fetcher != null) {
+				fetcher.update();
+				MetricStore.SubtaskMetricStore metrics = fetcher.getMetricStore().getSubtaskMetricStore(jobID, taskID, attempt.getParallelSubtaskIndex());
+				if (metrics != null) {
+					this.numBytesInLocal += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_IN_LOCAL, "0"));
+					this.numBytesInRemote += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_IN_REMOTE, "0"));
+					this.numBytesOut += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_OUT, "0"));
+					this.numRecordsIn += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_RECORDS_IN, "0"));
+					this.numRecordsOut += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_RECORDS_OUT, "0"));
+				}
+			}
+		}
+	}
+
+	/**
+	 * Writes the IO metrics contained in this object to the given {@link JsonGenerator}.
+	 *
+	 * <p>The JSON structure written is as follows:
+	 * "metrics": {
+	 *     "read-bytes": 1,
+	 *     "write-bytes": 2,
+	 *     "read-records": 3,
+	 *     "write-records": 4
+	 * }
+	 *
+	 * @param gen JsonGenerator to which the metrics should be written
+	 * @throws IOException
+	 */
+	public void writeIOMetricsAsJson(JsonGenerator gen) throws IOException {
+		gen.writeObjectFieldStart("metrics");
+		gen.writeNumberField("read-bytes", this.numBytesInLocal + this.numBytesInRemote);
+		gen.writeNumberField("write-bytes", this.numBytesOut);
+		gen.writeNumberField("read-records", this.numRecordsIn);
+		gen.writeNumberField("write-records", this.numRecordsOut);
+		gen.writeEndObject();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandlerTest.java
new file mode 100644
index 0000000..5bfa1f9
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandlerTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Executors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the ClusterOverviewHandler.
+ */
+public class ClusterOverviewHandlerTest {
+	@Test
+	public void testGetPaths() {
+		ClusterOverviewHandler handler = new ClusterOverviewHandler(Executors.directExecutor(), Time.seconds(0L));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/overview", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java
new file mode 100644
index 0000000..0ada30d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Executors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the CurrentJobIdsHandler.
+ */
+public class CurrentJobIdsHandlerTest {
+	@Test
+	public void testGetPaths() {
+		CurrentJobIdsHandler handler = new CurrentJobIdsHandler(Executors.directExecutor(), Time.seconds(0L));
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandlerTest.java
new file mode 100644
index 0000000..83bb157
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobsOverviewHandlerTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+
+/**
+ * Tests for the CurrentJobsOverviewHandler.
+ */
+public class CurrentJobsOverviewHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new CurrentJobsOverviewHandler.CurrentJobsOverviewJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		JobDetails expectedDetails = WebMonitorUtils.createDetailsForJob(originalJob);
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/joboverview", archive.getPath());
+
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(archive.getJson());
+		ArrayNode running = (ArrayNode) result.get("running");
+		Assert.assertEquals(0, running.size());
+
+		ArrayNode finished = (ArrayNode) result.get("finished");
+		Assert.assertEquals(1, finished.size());
+
+		compareJobOverview(expectedDetails, finished.get(0).toString());
+	}
+
+	@Test
+	public void testGetPaths() {
+		CurrentJobsOverviewHandler handlerAll = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, true);
+		String[] pathsAll = handlerAll.getPaths();
+		Assert.assertEquals(1, pathsAll.length);
+		Assert.assertEquals("/joboverview", pathsAll[0]);
+
+		CurrentJobsOverviewHandler handlerRunning = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, false);
+		String[] pathsRunning = handlerRunning.getPaths();
+		Assert.assertEquals(1, pathsRunning.length);
+		Assert.assertEquals("/joboverview/running", pathsRunning[0]);
+
+		CurrentJobsOverviewHandler handlerCompleted = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), false, true);
+		String[] pathsCompleted = handlerCompleted.getPaths();
+		Assert.assertEquals(1, pathsCompleted.length);
+		Assert.assertEquals("/joboverview/completed", pathsCompleted[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		JobDetails expectedDetails = WebMonitorUtils.createDetailsForJob(originalJob);
+		StringWriter writer = new StringWriter();
+		try (JsonGenerator gen = ArchivedJobGenerationUtils.JACKSON_FACTORY.createGenerator(writer)) {
+			CurrentJobsOverviewHandler.writeJobDetailOverviewAsJson(expectedDetails, gen, 0);
+		}
+		compareJobOverview(expectedDetails, writer.toString());
+	}
+
+	private static void compareJobOverview(JobDetails expectedDetails, String answer) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
+
+		Assert.assertEquals(expectedDetails.getJobId().toString(), result.get("jid").asText());
+		Assert.assertEquals(expectedDetails.getJobName(), result.get("name").asText());
+		Assert.assertEquals(expectedDetails.getStatus().name(), result.get("state").asText());
+
+		Assert.assertEquals(expectedDetails.getStartTime(), result.get("start-time").asLong());
+		Assert.assertEquals(expectedDetails.getEndTime(), result.get("end-time").asLong());
+		Assert.assertEquals(expectedDetails.getEndTime() - expectedDetails.getStartTime(), result.get("duration").asLong());
+		Assert.assertEquals(expectedDetails.getLastUpdateTime(), result.get("last-modification").asLong());
+
+		JsonNode tasks = result.get("tasks");
+		Assert.assertEquals(expectedDetails.getNumTasks(), tasks.get("total").asInt());
+		int[] tasksPerState = expectedDetails.getNumVerticesPerExecutionState();
+		Assert.assertEquals(
+			tasksPerState[ExecutionState.CREATED.ordinal()] + tasksPerState[ExecutionState.SCHEDULED.ordinal()] + tasksPerState[ExecutionState.DEPLOYING.ordinal()],
+			tasks.get("pending").asInt());
+		Assert.assertEquals(tasksPerState[ExecutionState.RUNNING.ordinal()], tasks.get("running").asInt());
+		Assert.assertEquals(tasksPerState[ExecutionState.FINISHED.ordinal()], tasks.get("finished").asInt());
+		Assert.assertEquals(tasksPerState[ExecutionState.CANCELING.ordinal()], tasks.get("canceling").asInt());
+		Assert.assertEquals(tasksPerState[ExecutionState.CANCELED.ordinal()], tasks.get("canceled").asInt());
+		Assert.assertEquals(tasksPerState[ExecutionState.FAILED.ordinal()], tasks.get("failed").asInt());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandlerTest.java
new file mode 100644
index 0000000..06a99fe
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandlerTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.TimeZone;
+
+/**
+ * Tests for the DashboardConfigHandler.
+ */
+public class DashboardConfigHandlerTest {
+	@Test
+	public void testGetPaths() {
+		DashboardConfigHandler handler = new DashboardConfigHandler(Executors.directExecutor(), 10000L);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/config", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		long refreshInterval = 12345;
+		TimeZone timeZone = TimeZone.getDefault();
+		EnvironmentInformation.RevisionInformation revision = EnvironmentInformation.getRevisionInformation();
+
+		String json = DashboardConfigHandler.createConfigJson(refreshInterval);
+
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(refreshInterval, result.get("refresh-interval").asLong());
+		Assert.assertEquals(timeZone.getDisplayName(), result.get("timezone-name").asText());
+		Assert.assertEquals(timeZone.getRawOffset(), result.get("timezone-offset").asLong());
+		Assert.assertEquals(EnvironmentInformation.getVersion(), result.get("flink-version").asText());
+		Assert.assertEquals(revision.commitId + " @ " + revision.commitDate, result.get("flink-revision").asText());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/HandlerRedirectUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/HandlerRedirectUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/HandlerRedirectUtilsTest.java
new file mode 100644
index 0000000..7e96835
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/HandlerRedirectUtilsTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.util.HandlerRedirectUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the HandlerRedirectUtils.
+ */
+public class HandlerRedirectUtilsTest extends TestLogger {
+
+	private static final String localRestAddress = "http://127.0.0.1:1234";
+	private static final String remoteRestAddress = "http://127.0.0.2:1234";
+
+	@Test
+	public void testGetRedirectAddressWithLocalEqualsRemoteRESTAddress() throws Exception {
+		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
+		when(jobManagerGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(localRestAddress));
+
+		CompletableFuture<Optional<String>> redirectingAddressFuture = HandlerRedirectUtils.getRedirectAddress(
+			localRestAddress,
+			jobManagerGateway,
+			Time.seconds(3L));
+
+		Assert.assertTrue(redirectingAddressFuture.isDone());
+		// no redirection needed
+		Assert.assertFalse(redirectingAddressFuture.get().isPresent());
+	}
+
+	@Test
+	public void testGetRedirectAddressWithRemoteAkkaPath() throws Exception {
+		JobManagerGateway jobManagerGateway = mock(AkkaJobManagerGateway.class);
+		when(jobManagerGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(remoteRestAddress));
+
+		CompletableFuture<Optional<String>> optRedirectingAddress = HandlerRedirectUtils.getRedirectAddress(
+			localRestAddress,
+			jobManagerGateway,
+			Time.seconds(3L));
+
+		Assert.assertTrue(optRedirectingAddress.isDone());
+
+		Assert.assertEquals(remoteRestAddress, optRedirectingAddress.get().get());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandlerTest.java
new file mode 100644
index 0000000..e1736c1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobAccumulatorsHandlerTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobAccumulatorsHandler.
+ */
+public class JobAccumulatorsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobAccumulatorsHandler.JobAccumulatorsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/accumulators", archive.getPath());
+		compareAccumulators(originalJob, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobAccumulatorsHandler handler = new JobAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/accumulators", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		String json = JobAccumulatorsHandler.createJobAccumulatorsJson(originalJob);
+
+		compareAccumulators(originalJob, json);
+	}
+
+	private static void compareAccumulators(AccessExecutionGraph originalJob, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		ArrayNode accs = (ArrayNode) result.get("job-accumulators");
+		Assert.assertEquals(0, accs.size());
+
+		Assert.assertTrue(originalJob.getAccumulatorResultsStringified().length > 0);
+		ArchivedJobGenerationUtils.compareStringifiedAccumulators(
+			originalJob.getAccumulatorResultsStringified(),
+			(ArrayNode) result.get("user-task-accumulators"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandlerTest.java
new file mode 100644
index 0000000..cab8835
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationHandlerTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Tests for the JobCancellationHandler.
+ */
+public class JobCancellationHandlerTest {
+	@Test
+	public void testGetPaths() {
+		JobCancellationHandler handler = new JobCancellationHandler(Executors.directExecutor(), TestingUtils.TIMEOUT());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(2, paths.length);
+		List<String> pathsList = Lists.newArrayList(paths);
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid/cancel"));
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid/yarn-cancel"));
+	}
+}


[13/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index 9f83ed0..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
+++ /dev/null
@@ -1,205 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.metrics.MetricStore;
-import org.apache.flink.util.StringUtils;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * A request handler that provides an overview over all taskmanagers or details for a single one.
- */
-public class TaskManagersHandler extends AbstractJsonRequestHandler  {
-
-	private static final String TASKMANAGERS_REST_PATH = "/taskmanagers";
-	private static final String TASKMANAGER_DETAILS_REST_PATH = "/taskmanagers/:taskmanagerid";
-
-	public static final String TASK_MANAGER_ID_KEY = "taskmanagerid";
-
-	private final Time timeout;
-
-	private final MetricFetcher fetcher;
-
-	public TaskManagersHandler(Executor executor, Time timeout, MetricFetcher fetcher) {
-		super(executor);
-		this.timeout = requireNonNull(timeout);
-		this.fetcher = fetcher;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		if (jobManagerGateway != null) {
-			// whether one task manager's metrics are requested, or all task manager, we
-			// return them in an array. This avoids unnecessary code complexity.
-			// If only one task manager is requested, we only fetch one task manager metrics.
-			if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) {
-				InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(pathParams.get(TASK_MANAGER_ID_KEY)));
-				CompletableFuture<Optional<Instance>> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout);
-
-				return tmInstanceFuture.thenApplyAsync(
-					(Optional<Instance> optTaskManager) -> {
-						try {
-							return writeTaskManagersJson(
-								optTaskManager.map(Collections::singleton).orElse(Collections.emptySet()),
-								pathParams);
-						} catch (IOException e) {
-							throw new FlinkFutureException("Could not write TaskManagers JSON.", e);
-						}
-					},
-					executor);
-			} else {
-				CompletableFuture<Collection<Instance>> tmInstancesFuture = jobManagerGateway.requestTaskManagerInstances(timeout);
-
-				return tmInstancesFuture.thenApplyAsync(
-					(Collection<Instance> taskManagers) -> {
-						try {
-							return writeTaskManagersJson(taskManagers, pathParams);
-						} catch (IOException e) {
-							throw new FlinkFutureException("Could not write TaskManagers JSON.", e);
-						}
-					},
-					executor);
-			}
-		}
-		else {
-			return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
-		}
-	}
-
-	private String writeTaskManagersJson(Collection<Instance> instances, Map<String, String> pathParams) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-		gen.writeArrayFieldStart("taskmanagers");
-
-		for (Instance instance : instances) {
-			gen.writeStartObject();
-			gen.writeStringField("id", instance.getId().toString());
-			gen.writeStringField("path", instance.getTaskManagerGateway().getAddress());
-			gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort());
-			gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat());
-			gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots());
-			gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots());
-			gen.writeNumberField("cpuCores", instance.getResources().getNumberOfCPUCores());
-			gen.writeNumberField("physicalMemory", instance.getResources().getSizeOfPhysicalMemory());
-			gen.writeNumberField("freeMemory", instance.getResources().getSizeOfJvmHeap());
-			gen.writeNumberField("managedMemory", instance.getResources().getSizeOfManagedMemory());
-
-			// only send metrics when only one task manager requests them.
-			if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) {
-				fetcher.update();
-				MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString());
-				if (metrics != null) {
-					gen.writeObjectFieldStart("metrics");
-					long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
-					long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
-					long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));
-
-					gen.writeNumberField("heapCommitted", heapCommitted);
-					gen.writeNumberField("heapUsed", heapUsed);
-					gen.writeNumberField("heapMax", heapTotal);
-
-					long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
-					long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
-					long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));
-
-					gen.writeNumberField("nonHeapCommitted", nonHeapCommitted);
-					gen.writeNumberField("nonHeapUsed", nonHeapUsed);
-					gen.writeNumberField("nonHeapMax", nonHeapTotal);
-
-					gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted);
-					gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed);
-					gen.writeNumberField("totalMax", heapTotal + nonHeapTotal);
-
-					long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0"));
-					long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0"));
-					long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0"));
-
-					gen.writeNumberField("directCount", directCount);
-					gen.writeNumberField("directUsed", directUsed);
-					gen.writeNumberField("directMax", directMax);
-
-					long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0"));
-					long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0"));
-					long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0"));
-
-					gen.writeNumberField("mappedCount", mappedCount);
-					gen.writeNumberField("mappedUsed", mappedUsed);
-					gen.writeNumberField("mappedMax", mappedMax);
-
-					long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0"));
-					long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0"));
-
-					gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable);
-					gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal);
-
-					gen.writeArrayFieldStart("garbageCollectors");
-
-					for (String gcName : metrics.garbageCollectorNames) {
-						String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null);
-						String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null);
-						if (count != null  && time != null) {
-							gen.writeStartObject();
-							gen.writeStringField("name", gcName);
-							gen.writeNumberField("count", Long.valueOf(count));
-							gen.writeNumberField("time", Long.valueOf(time));
-							gen.writeEndObject();
-						}
-					}
-
-					gen.writeEndArray();
-					gen.writeEndObject();
-				}
-			}
-
-			gen.writeEndObject();
-		}
-
-		gen.writeEndArray();
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java
deleted file mode 100644
index 3affd7c..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java
+++ /dev/null
@@ -1,120 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
-import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Handler that returns a job's snapshotting settings.
- */
-public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String CHECKPOINT_CONFIG_REST_PATH = "/jobs/:jobid/checkpoints/config";
-
-	public CheckpointConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CHECKPOINT_CONFIG_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createCheckpointConfigJson(graph);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create checkpoint config json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the CheckpointConfigHandler.
-	 */
-	public static class CheckpointConfigJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createCheckpointConfigJson(graph);
-			String path = CHECKPOINT_CONFIG_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-
-	private static String createCheckpointConfigJson(AccessExecutionGraph graph) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-		JobCheckpointingSettings settings = graph.getJobCheckpointingSettings();
-
-		if (settings == null) {
-			return "{}";
-		}
-
-		gen.writeStartObject();
-		{
-			gen.writeStringField("mode", settings.isExactlyOnce() ? "exactly_once" : "at_least_once");
-			gen.writeNumberField("interval", settings.getCheckpointInterval());
-			gen.writeNumberField("timeout", settings.getCheckpointTimeout());
-			gen.writeNumberField("min_pause", settings.getMinPauseBetweenCheckpoints());
-			gen.writeNumberField("max_concurrent", settings.getMaxConcurrentCheckpoints());
-
-			ExternalizedCheckpointSettings externalization = settings.getExternalizedCheckpointSettings();
-			gen.writeObjectFieldStart("externalization");
-			{
-				if (externalization.externalizeCheckpoints()) {
-					gen.writeBooleanField("enabled", true);
-					gen.writeBooleanField("delete_on_cancellation", externalization.deleteOnCancellation());
-				} else {
-					gen.writeBooleanField("enabled", false);
-				}
-			}
-			gen.writeEndObject();
-
-		}
-		gen.writeEndObject();
-
-		gen.close();
-
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCache.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCache.java
deleted file mode 100644
index 974364d..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCache.java
+++ /dev/null
@@ -1,81 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-
-import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
-import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
-
-import javax.annotation.Nullable;
-
-/**
- * A size-based cache of accessed checkpoints for completed and failed
- * checkpoints.
- *
- * <p>Having this cache in place for accessed stats improves the user
- * experience quite a bit as accessed checkpoint stats stay available
- * and don't expire. For example if you manage to click on the last
- * checkpoint in the history, it is not available via the stats as soon
- * as another checkpoint is triggered. With the cache in place, the
- * checkpoint will still be available for investigation.
- */
-public class CheckpointStatsCache {
-
-	@Nullable
-	private final Cache<Long, AbstractCheckpointStats> cache;
-
-	public CheckpointStatsCache(int maxNumEntries) {
-		if (maxNumEntries > 0) {
-			this.cache = CacheBuilder.<Long, AbstractCheckpointStats>newBuilder()
-				.maximumSize(maxNumEntries)
-				.build();
-		} else {
-			this.cache = null;
-		}
-	}
-
-	/**
-	 * Try to add the checkpoint to the cache.
-	 *
-	 * @param checkpoint Checkpoint to be added.
-	 */
-	void tryAdd(AbstractCheckpointStats checkpoint) {
-		// Don't add in progress checkpoints as they will be replaced by their
-		// completed/failed version eventually.
-		if (cache != null && checkpoint != null && !checkpoint.getStatus().isInProgress()) {
-			cache.put(checkpoint.getCheckpointId(), checkpoint);
-		}
-	}
-
-	/**
-	 * Try to look up a checkpoint by it's ID in the cache.
-	 *
-	 * @param checkpointId ID of the checkpoint to look up.
-	 * @return The checkpoint or <code>null</code> if checkpoint not found.
-	 */
-	AbstractCheckpointStats tryGet(long checkpointId) {
-		if (cache != null) {
-			return cache.getIfPresent(checkpointId);
-		} else {
-			return null;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java
deleted file mode 100644
index 96cc3e0..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java
+++ /dev/null
@@ -1,203 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.TaskStateStats;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns checkpoint stats for a single job vertex.
- */
-public class CheckpointStatsDetailsHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String CHECKPOINT_STATS_DETAILS_REST_PATH = "/jobs/:jobid/checkpoints/details/:checkpointid";
-
-	private final CheckpointStatsCache cache;
-
-	public CheckpointStatsDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) {
-		super(executionGraphHolder, executor);
-		this.cache = cache;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CHECKPOINT_STATS_DETAILS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				long checkpointId = parseCheckpointId(params);
-				if (checkpointId == -1) {
-					return "{}";
-				}
-
-				CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
-				if (snapshot == null) {
-					return "{}";
-				}
-
-				AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId);
-
-				if (checkpoint != null) {
-					cache.tryAdd(checkpoint);
-				} else {
-					checkpoint = cache.tryGet(checkpointId);
-
-					if (checkpoint == null) {
-						return "{}";
-					}
-				}
-
-				try {
-					return createCheckpointDetailsJson(checkpoint);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create checkpoint details json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the CheckpointStatsDetails.
-	 */
-	public static class CheckpointStatsDetailsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			CheckpointStatsSnapshot stats = graph.getCheckpointStatsSnapshot();
-			if (stats == null) {
-				return Collections.emptyList();
-			}
-			CheckpointStatsHistory history = stats.getHistory();
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
-				String json = createCheckpointDetailsJson(checkpoint);
-				String path = CHECKPOINT_STATS_DETAILS_REST_PATH
-					.replace(":jobid", graph.getJobID().toString())
-					.replace(":checkpointid", String.valueOf(checkpoint.getCheckpointId()));
-				archive.add(new ArchivedJson(path, json));
-			}
-			return archive;
-		}
-	}
-
-	public static String createCheckpointDetailsJson(AbstractCheckpointStats checkpoint) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-		gen.writeStartObject();
-
-		gen.writeNumberField("id", checkpoint.getCheckpointId());
-		gen.writeStringField("status", checkpoint.getStatus().toString());
-		gen.writeBooleanField("is_savepoint", checkpoint.getProperties().isSavepoint());
-		gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
-		gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
-		gen.writeNumberField("state_size", checkpoint.getStateSize());
-		gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
-		gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
-		gen.writeNumberField("num_subtasks", checkpoint.getNumberOfSubtasks());
-		gen.writeNumberField("num_acknowledged_subtasks", checkpoint.getNumberOfAcknowledgedSubtasks());
-
-		if (checkpoint.getStatus().isCompleted()) {
-			// --- Completed ---
-			CompletedCheckpointStats completed = (CompletedCheckpointStats) checkpoint;
-
-			String externalPath = completed.getExternalPath();
-			if (externalPath != null) {
-				gen.writeStringField("external_path", externalPath);
-			}
-
-			gen.writeBooleanField("discarded", completed.isDiscarded());
-		}
-		else if (checkpoint.getStatus().isFailed()) {
-			// --- Failed ---
-			FailedCheckpointStats failed = (FailedCheckpointStats) checkpoint;
-
-			gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
-
-			String failureMsg = failed.getFailureMessage();
-			if (failureMsg != null) {
-				gen.writeStringField("failure_message", failureMsg);
-			}
-		}
-
-		gen.writeObjectFieldStart("tasks");
-		for (TaskStateStats taskStats : checkpoint.getAllTaskStateStats()) {
-			gen.writeObjectFieldStart(taskStats.getJobVertexId().toString());
-
-			gen.writeNumberField("latest_ack_timestamp", taskStats.getLatestAckTimestamp());
-			gen.writeNumberField("state_size", taskStats.getStateSize());
-			gen.writeNumberField("end_to_end_duration", taskStats.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
-			gen.writeNumberField("alignment_buffered", taskStats.getAlignmentBuffered());
-			gen.writeNumberField("num_subtasks", taskStats.getNumberOfSubtasks());
-			gen.writeNumberField("num_acknowledged_subtasks", taskStats.getNumberOfAcknowledgedSubtasks());
-
-			gen.writeEndObject();
-		}
-		gen.writeEndObject();
-
-		gen.writeEndObject();
-		gen.close();
-
-		return writer.toString();
-	}
-
-	/**
-	 * Returns the checkpoint ID parsed from the provided parameters.
-	 *
-	 * @param params Path parameters
-	 * @return Parsed checkpoint ID or <code>-1</code> if not available.
-	 */
-	static long parseCheckpointId(Map<String, String> params) {
-		String param = params.get("checkpointid");
-		if (param == null) {
-			return -1;
-		}
-
-		try {
-			return Long.parseLong(param);
-		} catch (NumberFormatException ignored) {
-			return -1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
deleted file mode 100644
index 045248b..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java
+++ /dev/null
@@ -1,234 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
-import org.apache.flink.runtime.checkpoint.SubtaskStateStats;
-import org.apache.flink.runtime.checkpoint.TaskStateStats;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractJobVertexRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsHandler.writeMinMaxAvg;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Request handler that returns checkpoint stats for a single job vertex with
- * the summary stats and all subtasks.
- */
-public class CheckpointStatsDetailsSubtasksHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH = "/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid";
-
-	private final CheckpointStatsCache cache;
-
-	public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) {
-		super(executionGraphHolder, executor);
-		this.cache = checkNotNull(cache);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(
-			Map<String, String> pathParams,
-			Map<String, String> queryParams,
-			JobManagerGateway jobManagerGateway) {
-		return super.handleJsonRequest(pathParams, queryParams, jobManagerGateway);
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		long checkpointId = CheckpointStatsDetailsHandler.parseCheckpointId(params);
-		if (checkpointId == -1) {
-			return CompletableFuture.completedFuture("{}");
-		}
-
-		JobVertexID vertexId = AbstractJobVertexRequestHandler.parseJobVertexId(params);
-		if (vertexId == null) {
-			return CompletableFuture.completedFuture("{}");
-		}
-
-		CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
-		if (snapshot == null) {
-			return CompletableFuture.completedFuture("{}");
-		}
-
-		AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId);
-
-		if (checkpoint != null) {
-			cache.tryAdd(checkpoint);
-		} else {
-			checkpoint = cache.tryGet(checkpointId);
-
-			if (checkpoint == null) {
-				return CompletableFuture.completedFuture("{}");
-			}
-		}
-
-		TaskStateStats taskStats = checkpoint.getTaskStateStats(vertexId);
-		if (taskStats == null) {
-			return CompletableFuture.completedFuture("{}");
-		}
-
-		try {
-			return CompletableFuture.completedFuture(createSubtaskCheckpointDetailsJson(checkpoint, taskStats));
-		} catch (IOException e) {
-			return FutureUtils.completedExceptionally(e);
-		}
-	}
-
-	/**
-	 * Archivist for the CheckpointStatsDetailsSubtasksHandler.
-	 */
-	public static class CheckpointStatsDetailsSubtasksJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			CheckpointStatsSnapshot stats = graph.getCheckpointStatsSnapshot();
-			if (stats == null) {
-				return Collections.emptyList();
-			}
-			CheckpointStatsHistory history = stats.getHistory();
-			List<ArchivedJson> archive = new ArrayList<>();
-			for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
-				for (TaskStateStats subtaskStats : checkpoint.getAllTaskStateStats()) {
-					String json = createSubtaskCheckpointDetailsJson(checkpoint, subtaskStats);
-					String path = CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH
-						.replace(":jobid", graph.getJobID().toString())
-						.replace(":checkpointid", String.valueOf(checkpoint.getCheckpointId()))
-						.replace(":vertexid", subtaskStats.getJobVertexId().toString());
-					archive.add(new ArchivedJson(path, json));
-				}
-			}
-			return archive;
-		}
-	}
-
-	private static String createSubtaskCheckpointDetailsJson(AbstractCheckpointStats checkpoint, TaskStateStats taskStats) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-		// Overview
-		gen.writeNumberField("id", checkpoint.getCheckpointId());
-		gen.writeStringField("status", checkpoint.getStatus().toString());
-		gen.writeNumberField("latest_ack_timestamp", taskStats.getLatestAckTimestamp());
-		gen.writeNumberField("state_size", taskStats.getStateSize());
-		gen.writeNumberField("end_to_end_duration", taskStats.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
-		gen.writeNumberField("alignment_buffered", taskStats.getAlignmentBuffered());
-		gen.writeNumberField("num_subtasks", taskStats.getNumberOfSubtasks());
-		gen.writeNumberField("num_acknowledged_subtasks", taskStats.getNumberOfAcknowledgedSubtasks());
-
-		if (taskStats.getNumberOfAcknowledgedSubtasks() > 0) {
-			gen.writeObjectFieldStart("summary");
-			gen.writeObjectFieldStart("state_size");
-			writeMinMaxAvg(gen, taskStats.getSummaryStats().getStateSizeStats());
-			gen.writeEndObject();
-
-			gen.writeObjectFieldStart("end_to_end_duration");
-			MinMaxAvgStats ackTimestampStats = taskStats.getSummaryStats().getAckTimestampStats();
-			gen.writeNumberField("min", Math.max(0, ackTimestampStats.getMinimum() - checkpoint.getTriggerTimestamp()));
-			gen.writeNumberField("max", Math.max(0, ackTimestampStats.getMaximum() - checkpoint.getTriggerTimestamp()));
-			gen.writeNumberField("avg", Math.max(0, ackTimestampStats.getAverage() - checkpoint.getTriggerTimestamp()));
-			gen.writeEndObject();
-
-			gen.writeObjectFieldStart("checkpoint_duration");
-			gen.writeObjectFieldStart("sync");
-			writeMinMaxAvg(gen, taskStats.getSummaryStats().getSyncCheckpointDurationStats());
-			gen.writeEndObject();
-			gen.writeObjectFieldStart("async");
-			writeMinMaxAvg(gen, taskStats.getSummaryStats().getAsyncCheckpointDurationStats());
-			gen.writeEndObject();
-			gen.writeEndObject();
-
-			gen.writeObjectFieldStart("alignment");
-			gen.writeObjectFieldStart("buffered");
-			writeMinMaxAvg(gen, taskStats.getSummaryStats().getAlignmentBufferedStats());
-			gen.writeEndObject();
-			gen.writeObjectFieldStart("duration");
-			writeMinMaxAvg(gen, taskStats.getSummaryStats().getAlignmentDurationStats());
-			gen.writeEndObject();
-			gen.writeEndObject();
-			gen.writeEndObject();
-		}
-
-		SubtaskStateStats[] subtasks = taskStats.getSubtaskStats();
-
-		gen.writeArrayFieldStart("subtasks");
-		for (int i = 0; i < subtasks.length; i++) {
-			SubtaskStateStats subtask = subtasks[i];
-
-			gen.writeStartObject();
-			gen.writeNumberField("index", i);
-
-			if (subtask != null) {
-				gen.writeStringField("status", "completed");
-				gen.writeNumberField("ack_timestamp", subtask.getAckTimestamp());
-				gen.writeNumberField("end_to_end_duration", subtask.getEndToEndDuration(checkpoint.getTriggerTimestamp()));
-				gen.writeNumberField("state_size", subtask.getStateSize());
-
-				gen.writeObjectFieldStart("checkpoint");
-				gen.writeNumberField("sync", subtask.getSyncCheckpointDuration());
-				gen.writeNumberField("async", subtask.getAsyncCheckpointDuration());
-				gen.writeEndObject();
-
-				gen.writeObjectFieldStart("alignment");
-				gen.writeNumberField("buffered", subtask.getAlignmentBuffered());
-				gen.writeNumberField("duration", subtask.getAlignmentDuration());
-				gen.writeEndObject();
-			} else {
-				gen.writeStringField("status", "pending_or_failed");
-			}
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeEndObject();
-		gen.close();
-
-		return writer.toString();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java
deleted file mode 100644
index a60aee0..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java
+++ /dev/null
@@ -1,277 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStatsSummary;
-import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
-import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Handler that returns checkpoint statistics for a job.
- */
-public class CheckpointStatsHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String CHECKPOINT_STATS_REST_PATH = "/jobs/:jobid/checkpoints";
-
-	public CheckpointStatsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CHECKPOINT_STATS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createCheckpointStatsJson(graph);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create checkpoint stats json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the CheckpointStatsJsonHandler.
-	 */
-	public static class CheckpointStatsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createCheckpointStatsJson(graph);
-			String path = CHECKPOINT_STATS_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-
-	private static String createCheckpointStatsJson(AccessExecutionGraph graph) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
-		if (snapshot == null) {
-			return "{}";
-		}
-
-		gen.writeStartObject();
-
-		// Counts
-		writeCounts(gen, snapshot.getCounts());
-
-		// Summary
-		writeSummary(gen, snapshot.getSummaryStats());
-
-		CheckpointStatsHistory history = snapshot.getHistory();
-
-		// Latest
-		writeLatestCheckpoints(
-			gen,
-			history.getLatestCompletedCheckpoint(),
-			history.getLatestSavepoint(),
-			history.getLatestFailedCheckpoint(),
-			snapshot.getLatestRestoredCheckpoint());
-
-		// History
-		writeHistory(gen, snapshot.getHistory());
-
-		gen.writeEndObject();
-		gen.close();
-
-		return writer.toString();
-	}
-
-	private static void writeCounts(JsonGenerator gen, CheckpointStatsCounts counts) throws IOException {
-		gen.writeObjectFieldStart("counts");
-		gen.writeNumberField("restored", counts.getNumberOfRestoredCheckpoints());
-		gen.writeNumberField("total", counts.getTotalNumberOfCheckpoints());
-		gen.writeNumberField("in_progress", counts.getNumberOfInProgressCheckpoints());
-		gen.writeNumberField("completed", counts.getNumberOfCompletedCheckpoints());
-		gen.writeNumberField("failed", counts.getNumberOfFailedCheckpoints());
-		gen.writeEndObject();
-	}
-
-	private static void writeSummary(
-		JsonGenerator gen,
-		CompletedCheckpointStatsSummary summary) throws IOException {
-		gen.writeObjectFieldStart("summary");
-		gen.writeObjectFieldStart("state_size");
-		writeMinMaxAvg(gen, summary.getStateSizeStats());
-		gen.writeEndObject();
-
-		gen.writeObjectFieldStart("end_to_end_duration");
-		writeMinMaxAvg(gen, summary.getEndToEndDurationStats());
-		gen.writeEndObject();
-
-		gen.writeObjectFieldStart("alignment_buffered");
-		writeMinMaxAvg(gen, summary.getAlignmentBufferedStats());
-		gen.writeEndObject();
-		gen.writeEndObject();
-	}
-
-	static void writeMinMaxAvg(JsonGenerator gen, MinMaxAvgStats minMaxAvg) throws IOException {
-		gen.writeNumberField("min", minMaxAvg.getMinimum());
-		gen.writeNumberField("max", minMaxAvg.getMaximum());
-		gen.writeNumberField("avg", minMaxAvg.getAverage());
-	}
-
-	private static void writeLatestCheckpoints(
-		JsonGenerator gen,
-		@Nullable CompletedCheckpointStats completed,
-		@Nullable CompletedCheckpointStats savepoint,
-		@Nullable FailedCheckpointStats failed,
-		@Nullable RestoredCheckpointStats restored) throws IOException {
-
-		gen.writeObjectFieldStart("latest");
-		// Completed checkpoint
-		if (completed != null) {
-			gen.writeObjectFieldStart("completed");
-			writeCheckpoint(gen, completed);
-
-			String externalPath = completed.getExternalPath();
-			if (externalPath != null) {
-				gen.writeStringField("external_path", completed.getExternalPath());
-			}
-
-			gen.writeEndObject();
-		}
-
-		// Completed savepoint
-		if (savepoint != null) {
-			gen.writeObjectFieldStart("savepoint");
-			writeCheckpoint(gen, savepoint);
-
-			String externalPath = savepoint.getExternalPath();
-			if (externalPath != null) {
-				gen.writeStringField("external_path", savepoint.getExternalPath());
-			}
-			gen.writeEndObject();
-		}
-
-		// Failed checkpoint
-		if (failed != null) {
-			gen.writeObjectFieldStart("failed");
-			writeCheckpoint(gen, failed);
-
-			gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
-			String failureMsg = failed.getFailureMessage();
-			if (failureMsg != null) {
-				gen.writeStringField("failure_message", failureMsg);
-			}
-			gen.writeEndObject();
-		}
-
-		// Restored checkpoint
-		if (restored != null) {
-			gen.writeObjectFieldStart("restored");
-			gen.writeNumberField("id", restored.getCheckpointId());
-			gen.writeNumberField("restore_timestamp", restored.getRestoreTimestamp());
-			gen.writeBooleanField("is_savepoint", restored.getProperties().isSavepoint());
-
-			String externalPath = restored.getExternalPath();
-			if (externalPath != null) {
-				gen.writeStringField("external_path", externalPath);
-			}
-			gen.writeEndObject();
-		}
-		gen.writeEndObject();
-	}
-
-	private static void writeCheckpoint(JsonGenerator gen, AbstractCheckpointStats checkpoint) throws IOException {
-		gen.writeNumberField("id", checkpoint.getCheckpointId());
-		gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
-		gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
-		gen.writeNumberField("state_size", checkpoint.getStateSize());
-		gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
-		gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
-
-	}
-
-	private static void writeHistory(JsonGenerator gen, CheckpointStatsHistory history) throws IOException {
-		gen.writeArrayFieldStart("history");
-		for (AbstractCheckpointStats checkpoint : history.getCheckpoints()) {
-			gen.writeStartObject();
-			gen.writeNumberField("id", checkpoint.getCheckpointId());
-			gen.writeStringField("status", checkpoint.getStatus().toString());
-			gen.writeBooleanField("is_savepoint", checkpoint.getProperties().isSavepoint());
-			gen.writeNumberField("trigger_timestamp", checkpoint.getTriggerTimestamp());
-			gen.writeNumberField("latest_ack_timestamp", checkpoint.getLatestAckTimestamp());
-			gen.writeNumberField("state_size", checkpoint.getStateSize());
-			gen.writeNumberField("end_to_end_duration", checkpoint.getEndToEndDuration());
-			gen.writeNumberField("alignment_buffered", checkpoint.getAlignmentBuffered());
-			gen.writeNumberField("num_subtasks", checkpoint.getNumberOfSubtasks());
-			gen.writeNumberField("num_acknowledged_subtasks", checkpoint.getNumberOfAcknowledgedSubtasks());
-
-			if (checkpoint.getStatus().isCompleted()) {
-				// --- Completed ---
-				CompletedCheckpointStats completed = (CompletedCheckpointStats) checkpoint;
-
-				String externalPath = completed.getExternalPath();
-				if (externalPath != null) {
-					gen.writeStringField("external_path", externalPath);
-				}
-
-				gen.writeBooleanField("discarded", completed.isDiscarded());
-			}
-			else if (checkpoint.getStatus().isFailed()) {
-				// --- Failed ---
-				FailedCheckpointStats failed = (FailedCheckpointStats) checkpoint;
-
-				gen.writeNumberField("failure_timestamp", failed.getFailureTimestamp());
-
-				String failureMsg = failed.getFailureMessage();
-				if (failureMsg != null) {
-					gen.writeStringField("failure_message", failureMsg);
-				}
-			}
-
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
index 6da8115..01228d5 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
@@ -27,9 +27,9 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.history.FsJobArchivist;
 import org.apache.flink.runtime.net.SSLUtils;
+import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
-import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
 import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.FlinkException;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
index 0fc4314..bae8e21 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
@@ -24,8 +24,8 @@ import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.history.FsJobArchivist;
+import org.apache.flink.runtime.rest.handler.legacy.CurrentJobsOverviewHandler;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
-import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
 import org.apache.flink.util.FileUtils;
 
 import com.fasterxml.jackson.core.JsonFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
index c5943dc..12a27a7 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
@@ -26,7 +26,7 @@ package org.apache.flink.runtime.webmonitor.history;
  * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
  *****************************************************************************/
 
-import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
+import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
 
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java
deleted file mode 100644
index cf286ce..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java
+++ /dev/null
@@ -1,139 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.webmonitor.handlers.AbstractJsonRequestHandler;
-import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
-import org.apache.flink.util.Preconditions;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Abstract request handler that returns a list of all available metrics or the values for a set of metrics.
- *
- * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
- * {@code [ { "id" : "X" } ] }
- *
- * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
- * {@code /get?X,Y}
- * The handler will then return a list containing the values of the requested metrics.
- * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
- */
-public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler {
-	private final MetricFetcher fetcher;
-
-	public AbstractMetricsHandler(Executor executor, MetricFetcher fetcher) {
-		super(executor);
-		this.fetcher = Preconditions.checkNotNull(fetcher);
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				fetcher.update();
-				String requestedMetricsList = queryParams.get("get");
-				try {
-					return requestedMetricsList != null
-						? getMetricsValues(pathParams, requestedMetricsList)
-						: getAvailableMetricsList(pathParams);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not retrieve metrics.", e);
-				}
-			},
-			executor);
-
-	}
-
-	/**
-	 * Returns a Map containing the metrics belonging to the entity pointed to by the path parameters.
-	 *
-	 * @param pathParams REST path parameters
-	 * @param metrics MetricStore containing all metrics
-	 * @return Map containing metrics, or null if no metric exists
-	 */
-	protected abstract Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics);
-
-	private String getMetricsValues(Map<String, String> pathParams, String requestedMetricsList) throws IOException {
-		if (requestedMetricsList.isEmpty()) {
-			/*
-			 * The WebInterface doesn't check whether the list of available metrics was empty. This can lead to a
-			 * request for which the "get" parameter is an empty string.
-			 */
-			return "";
-		}
-		MetricStore metricStore = fetcher.getMetricStore();
-		synchronized (metricStore) {
-			Map<String, String> metrics = getMapFor(pathParams, metricStore);
-			if (metrics == null) {
-				return "";
-			}
-			String[] requestedMetrics = requestedMetricsList.split(",");
-
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-			gen.writeStartArray();
-			for (String requestedMetric : requestedMetrics) {
-				Object metricValue = metrics.get(requestedMetric);
-				if (metricValue != null) {
-					gen.writeStartObject();
-					gen.writeStringField("id", requestedMetric);
-					gen.writeStringField("value", metricValue.toString());
-					gen.writeEndObject();
-				}
-			}
-			gen.writeEndArray();
-
-			gen.close();
-			return writer.toString();
-		}
-	}
-
-	private String getAvailableMetricsList(Map<String, String> pathParams) throws IOException {
-		MetricStore metricStore = fetcher.getMetricStore();
-		synchronized (metricStore) {
-			Map<String, String> metrics = getMapFor(pathParams, metricStore);
-			if (metrics == null) {
-				return "";
-			}
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-			gen.writeStartArray();
-			for (String m : metrics.keySet()) {
-				gen.writeStartObject();
-				gen.writeStringField("id", m);
-				gen.writeEndObject();
-			}
-			gen.writeEndArray();
-
-			gen.close();
-			return writer.toString();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java
deleted file mode 100644
index 2bd6683..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java
+++ /dev/null
@@ -1,57 +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.webmonitor.metrics;
-
-import java.util.Map;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns for the job manager a list of all available metrics or the values for a set of metrics.
- *
- * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
- * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
- *
- * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
- * {@code /get?X,Y}
- * The handler will then return a list containing the values of the requested metrics.
- * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
- */
-public class JobManagerMetricsHandler extends AbstractMetricsHandler {
-
-	private static final String JOBMANAGER_METRICS_REST_PATH = "/jobmanager/metrics";
-
-	public JobManagerMetricsHandler(Executor executor, MetricFetcher fetcher) {
-		super(executor, fetcher);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOBMANAGER_METRICS_REST_PATH};
-	}
-
-	@Override
-	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
-		MetricStore.JobManagerMetricStore jobManager = metrics.getJobManagerMetricStore();
-		if (jobManager == null) {
-			return null;
-		} else {
-			return jobManager.metrics;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java
deleted file mode 100644
index e5e2500..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java
+++ /dev/null
@@ -1,55 +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.webmonitor.metrics;
-
-import java.util.Map;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns for a given job a list of all available metrics or the values for a set of metrics.
- *
- * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
- * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
- *
- * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
- * {@code /get?X,Y}
- * The handler will then return a list containing the values of the requested metrics.
- * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
- */
-public class JobMetricsHandler extends AbstractMetricsHandler {
-	public static final String PARAMETER_JOB_ID = "jobid";
-	private static final String JOB_METRICS_REST_PATH = "/jobs/:jobid/metrics";
-
-	public JobMetricsHandler(Executor executor, MetricFetcher fetcher) {
-		super(executor, fetcher);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_METRICS_REST_PATH};
-	}
-
-	@Override
-	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
-		MetricStore.JobMetricStore job = metrics.getJobMetricStore(pathParams.get(PARAMETER_JOB_ID));
-		return job != null
-			? job.metrics
-			: null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java
deleted file mode 100644
index 1d2cd84..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java
+++ /dev/null
@@ -1,57 +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.webmonitor.metrics;
-
-import java.util.Map;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns for a given task a list of all available metrics or the values for a set of metrics.
- *
- * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
- * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
- *
- * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
- * {@code /get?X,Y}
- * The handler will then return a list containing the values of the requested metrics.
- * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
- */
-public class JobVertexMetricsHandler extends AbstractMetricsHandler {
-	public static final String PARAMETER_VERTEX_ID = "vertexid";
-	private static final String JOB_VERTEX_METRICS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/metrics";
-
-	public JobVertexMetricsHandler(Executor executor, MetricFetcher fetcher) {
-		super(executor, fetcher);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_VERTEX_METRICS_REST_PATH};
-	}
-
-	@Override
-	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
-		MetricStore.TaskMetricStore task = metrics.getTaskMetricStore(
-			pathParams.get(JobMetricsHandler.PARAMETER_JOB_ID),
-			pathParams.get(PARAMETER_VERTEX_ID));
-		return task != null
-			? task.metrics
-			: null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java
deleted file mode 100644
index a5f4ca5..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java
+++ /dev/null
@@ -1,211 +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.webmonitor.metrics;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.webmonitor.JobDetails;
-import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
-import org.apache.flink.runtime.metrics.dump.MetricDump;
-import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
-import org.apache.flink.runtime.metrics.dump.MetricQueryService;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
-import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
-import org.apache.flink.util.Preconditions;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.MetricDumpDeserializer;
-
-/**
- * The MetricFetcher can be used to fetch metrics from the JobManager and all registered TaskManagers.
- *
- * <p>Metrics will only be fetched when {@link MetricFetcher#update()} is called, provided that a sufficient time since
- * the last call has passed.
- */
-public class MetricFetcher {
-	private static final Logger LOG = LoggerFactory.getLogger(MetricFetcher.class);
-
-	private final GatewayRetriever<JobManagerGateway> retriever;
-	private final MetricQueryServiceRetriever queryServiceRetriever;
-	private final Executor executor;
-	private final Time timeout;
-
-	private final MetricStore metrics = new MetricStore();
-	private final MetricDumpDeserializer deserializer = new MetricDumpDeserializer();
-
-	private long lastUpdateTime;
-
-	public MetricFetcher(
-			GatewayRetriever<JobManagerGateway> retriever,
-			MetricQueryServiceRetriever queryServiceRetriever,
-			Executor executor,
-			Time timeout) {
-		this.retriever = Preconditions.checkNotNull(retriever);
-		this.queryServiceRetriever = Preconditions.checkNotNull(queryServiceRetriever);
-		this.executor = Preconditions.checkNotNull(executor);
-		this.timeout = Preconditions.checkNotNull(timeout);
-	}
-
-	/**
-	 * Returns the MetricStore containing all stored metrics.
-	 *
-	 * @return MetricStore containing all stored metrics;
-	 */
-	public MetricStore getMetricStore() {
-		return metrics;
-	}
-
-	/**
-	 * This method can be used to signal this MetricFetcher that the metrics are still in use and should be updated.
-	 */
-	public void update() {
-		synchronized (this) {
-			long currentTime = System.currentTimeMillis();
-			if (currentTime - lastUpdateTime > 10000) { // 10 seconds have passed since the last update
-				lastUpdateTime = currentTime;
-				fetchMetrics();
-			}
-		}
-	}
-
-	private void fetchMetrics() {
-		try {
-			Optional<JobManagerGateway> optJobManagerGateway = retriever.getNow();
-			if (optJobManagerGateway.isPresent()) {
-				final JobManagerGateway jobManagerGateway = optJobManagerGateway.get();
-
-				/**
-				 * Remove all metrics that belong to a job that is not running and no longer archived.
-				 */
-				CompletableFuture<MultipleJobsDetails> jobDetailsFuture = jobManagerGateway.requestJobDetails(true, true, timeout);
-
-				jobDetailsFuture.whenCompleteAsync(
-					(MultipleJobsDetails jobDetails, Throwable throwable) -> {
-						if (throwable != null) {
-							LOG.debug("Fetching of JobDetails failed.", throwable);
-						} else {
-							ArrayList<String> toRetain = new ArrayList<>();
-							for (JobDetails job : jobDetails.getRunningJobs()) {
-								toRetain.add(job.getJobId().toString());
-							}
-							for (JobDetails job : jobDetails.getFinishedJobs()) {
-								toRetain.add(job.getJobId().toString());
-							}
-							synchronized (metrics) {
-								metrics.jobs.keySet().retainAll(toRetain);
-							}
-						}
-					},
-					executor);
-
-				String jobManagerPath = jobManagerGateway.getAddress();
-				String jmQueryServicePath = jobManagerPath.substring(0, jobManagerPath.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME;
-
-				retrieveAndQueryMetrics(jmQueryServicePath);
-
-				/**
-				 * We first request the list of all registered task managers from the job manager, and then
-				 * request the respective metric dump from each task manager.
-				 *
-				 * <p>All stored metrics that do not belong to a registered task manager will be removed.
-				 */
-				CompletableFuture<Collection<Instance>> taskManagersFuture = jobManagerGateway.requestTaskManagerInstances(timeout);
-
-				taskManagersFuture.whenCompleteAsync(
-					(Collection<Instance> taskManagers, Throwable throwable) -> {
-						if (throwable != null) {
-							LOG.debug("Fetching list of registered TaskManagers failed.", throwable);
-						} else {
-							List<String> activeTaskManagers = taskManagers.stream().map(
-								taskManagerInstance -> {
-									final String taskManagerAddress = taskManagerInstance.getTaskManagerGateway().getAddress();
-									final String tmQueryServicePath = taskManagerAddress.substring(0, taskManagerAddress.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME + "_" + taskManagerInstance.getTaskManagerID().getResourceIdString();
-
-									retrieveAndQueryMetrics(tmQueryServicePath);
-
-									return taskManagerInstance.getId().toString();
-								}).collect(Collectors.toList());
-
-							synchronized (metrics) {
-								metrics.taskManagers.keySet().retainAll(activeTaskManagers);
-							}
-						}
-					},
-					executor);
-			}
-		} catch (Exception e) {
-			LOG.warn("Exception while fetching metrics.", e);
-		}
-	}
-
-	/**
-	 * Retrieves and queries the specified QueryServiceGateway.
-	 *
-	 * @param queryServicePath specifying the QueryServiceGateway
-	 */
-	private void retrieveAndQueryMetrics(String queryServicePath) {
-		final CompletableFuture<MetricQueryServiceGateway> queryServiceGatewayFuture = queryServiceRetriever.retrieveService(queryServicePath);
-
-		queryServiceGatewayFuture.whenCompleteAsync(
-			(MetricQueryServiceGateway queryServiceGateway, Throwable t) -> {
-				if (t != null) {
-					LOG.debug("Could not retrieve QueryServiceGateway.", t);
-				} else {
-					queryMetrics(queryServiceGateway);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Query the metrics from the given QueryServiceGateway.
-	 *
-	 * @param queryServiceGateway to query for metrics
-	 */
-	private void queryMetrics(final MetricQueryServiceGateway queryServiceGateway) {
-		queryServiceGateway
-			.queryMetrics(timeout)
-			.whenCompleteAsync(
-				(MetricDumpSerialization.MetricSerializationResult result, Throwable t) -> {
-					if (t != null) {
-						LOG.debug("Fetching metrics failed.", t);
-					} else {
-						List<MetricDump> dumpedMetrics = deserializer.deserialize(result);
-						synchronized (metrics) {
-							for (MetricDump metric : dumpedMetrics) {
-								metrics.add(metric);
-							}
-						}
-					}
-				},
-				executor);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricStore.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricStore.java
deleted file mode 100644
index e36dca8..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricStore.java
+++ /dev/null
@@ -1,305 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.metrics.dump.MetricDump;
-import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_COUNTER;
-import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_GAUGE;
-import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_HISTOGRAM;
-import static org.apache.flink.runtime.metrics.dump.MetricDump.METRIC_CATEGORY_METER;
-import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_JM;
-import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_JOB;
-import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_OPERATOR;
-import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_TASK;
-import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY_TM;
-
-/**
- * Nested data-structure to store metrics.
- *
- * <p>This structure is not thread-safe.
- */
-public class MetricStore {
-	private static final Logger LOG = LoggerFactory.getLogger(MetricStore.class);
-
-	final JobManagerMetricStore jobManager = new JobManagerMetricStore();
-	final Map<String, TaskManagerMetricStore> taskManagers = new HashMap<>();
-	final Map<String, JobMetricStore> jobs = new HashMap<>();
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Adding metrics
-	// -----------------------------------------------------------------------------------------------------------------
-	public void add(MetricDump metric) {
-		try {
-			QueryScopeInfo info = metric.scopeInfo;
-			TaskManagerMetricStore tm;
-			JobMetricStore job;
-			TaskMetricStore task;
-			SubtaskMetricStore subtask;
-
-			String name = info.scope.isEmpty()
-				? metric.name
-				: info.scope + "." + metric.name;
-
-			if (name.isEmpty()) { // malformed transmission
-				return;
-			}
-
-			switch (info.getCategory()) {
-				case INFO_CATEGORY_JM:
-					addMetric(jobManager.metrics, name, metric);
-					break;
-				case INFO_CATEGORY_TM:
-					String tmID = ((QueryScopeInfo.TaskManagerQueryScopeInfo) info).taskManagerID;
-					tm = taskManagers.get(tmID);
-					if (tm == null) {
-						tm = new TaskManagerMetricStore();
-						taskManagers.put(tmID, tm);
-					}
-					if (name.contains("GarbageCollector")) {
-						String gcName = name.substring("Status.JVM.GarbageCollector.".length(), name.lastIndexOf('.'));
-						tm.addGarbageCollectorName(gcName);
-					}
-					addMetric(tm.metrics, name, metric);
-					break;
-				case INFO_CATEGORY_JOB:
-					QueryScopeInfo.JobQueryScopeInfo jobInfo = (QueryScopeInfo.JobQueryScopeInfo) info;
-					job = jobs.get(jobInfo.jobID);
-					if (job == null) {
-						job = new JobMetricStore();
-						jobs.put(jobInfo.jobID, job);
-					}
-					addMetric(job.metrics, name, metric);
-					break;
-				case INFO_CATEGORY_TASK:
-					QueryScopeInfo.TaskQueryScopeInfo taskInfo = (QueryScopeInfo.TaskQueryScopeInfo) info;
-					job = jobs.get(taskInfo.jobID);
-					if (job == null) {
-						job = new JobMetricStore();
-						jobs.put(taskInfo.jobID, job);
-					}
-					task = job.tasks.get(taskInfo.vertexID);
-					if (task == null) {
-						task = new TaskMetricStore();
-						job.tasks.put(taskInfo.vertexID, task);
-					}
-					subtask = task.subtasks.get(taskInfo.subtaskIndex);
-					if (subtask == null) {
-						subtask = new SubtaskMetricStore();
-						task.subtasks.put(taskInfo.subtaskIndex, subtask);
-					}
-					/**
-					 * The duplication is intended. Metrics scoped by subtask are useful for several job/task handlers,
-					 * while the WebInterface task metric queries currently do not account for subtasks, so we don't
-					 * divide by subtask and instead use the concatenation of subtask index and metric name as the name
-					 * for those.
-					 */
-					addMetric(subtask.metrics, name, metric);
-					addMetric(task.metrics, taskInfo.subtaskIndex + "." + name, metric);
-					break;
-				case INFO_CATEGORY_OPERATOR:
-					QueryScopeInfo.OperatorQueryScopeInfo operatorInfo = (QueryScopeInfo.OperatorQueryScopeInfo) info;
-					job = jobs.get(operatorInfo.jobID);
-					if (job == null) {
-						job = new JobMetricStore();
-						jobs.put(operatorInfo.jobID, job);
-					}
-					task = job.tasks.get(operatorInfo.vertexID);
-					if (task == null) {
-						task = new TaskMetricStore();
-						job.tasks.put(operatorInfo.vertexID, task);
-					}
-					/**
-					 * As the WebInterface does not account for operators (because it can't) we don't
-					 * divide by operator and instead use the concatenation of subtask index, operator name and metric name
-					 * as the name.
-					 */
-					addMetric(task.metrics, operatorInfo.subtaskIndex + "." + operatorInfo.operatorName + "." + name, metric);
-					break;
-				default:
-					LOG.debug("Invalid metric dump category: " + info.getCategory());
-			}
-		} catch (Exception e) {
-			LOG.debug("Malformed metric dump.", e);
-		}
-	}
-
-	private void addMetric(Map<String, String> target, String name, MetricDump metric) {
-		switch (metric.getCategory()) {
-			case METRIC_CATEGORY_COUNTER:
-				MetricDump.CounterDump counter = (MetricDump.CounterDump) metric;
-				target.put(name, String.valueOf(counter.count));
-				break;
-			case METRIC_CATEGORY_GAUGE:
-				MetricDump.GaugeDump gauge = (MetricDump.GaugeDump) metric;
-				target.put(name, gauge.value);
-				break;
-			case METRIC_CATEGORY_HISTOGRAM:
-				MetricDump.HistogramDump histogram = (MetricDump.HistogramDump) metric;
-				target.put(name + "_min", String.valueOf(histogram.min));
-				target.put(name + "_max", String.valueOf(histogram.max));
-				target.put(name + "_mean", String.valueOf(histogram.mean));
-				target.put(name + "_median", String.valueOf(histogram.median));
-				target.put(name + "_stddev", String.valueOf(histogram.stddev));
-				target.put(name + "_p75", String.valueOf(histogram.p75));
-				target.put(name + "_p90", String.valueOf(histogram.p90));
-				target.put(name + "_p95", String.valueOf(histogram.p95));
-				target.put(name + "_p98", String.valueOf(histogram.p98));
-				target.put(name + "_p99", String.valueOf(histogram.p99));
-				target.put(name + "_p999", String.valueOf(histogram.p999));
-				break;
-			case METRIC_CATEGORY_METER:
-				MetricDump.MeterDump meter = (MetricDump.MeterDump) metric;
-				target.put(name, String.valueOf(meter.rate));
-				break;
-		}
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Accessors for sub MetricStores
-	// -----------------------------------------------------------------------------------------------------------------
-
-	/**
-	 * Returns the {@link JobManagerMetricStore}.
-	 *
-	 * @return JobManagerMetricStore
-	 */
-	public JobManagerMetricStore getJobManagerMetricStore() {
-		return jobManager;
-	}
-
-	/**
-	 * Returns the {@link TaskManagerMetricStore} for the given taskmanager ID.
-	 *
-	 * @param tmID taskmanager ID
-	 * @return TaskManagerMetricStore for the given ID, or null if no store for the given argument exists
-	 */
-	public TaskManagerMetricStore getTaskManagerMetricStore(String tmID) {
-		return taskManagers.get(tmID);
-	}
-
-	/**
-	 * Returns the {@link JobMetricStore} for the given job ID.
-	 *
-	 * @param jobID job ID
-	 * @return JobMetricStore for the given ID, or null if no store for the given argument exists
-	 */
-	public JobMetricStore getJobMetricStore(String jobID) {
-		return jobs.get(jobID);
-	}
-
-	/**
-	 * Returns the {@link TaskMetricStore} for the given job/task ID.
-	 *
-	 * @param jobID  job ID
-	 * @param taskID task ID
-	 * @return TaskMetricStore for given IDs, or null if no store for the given arguments exists
-	 */
-	public TaskMetricStore getTaskMetricStore(String jobID, String taskID) {
-		JobMetricStore job = getJobMetricStore(jobID);
-		if (job == null) {
-			return null;
-		}
-		return job.getTaskMetricStore(taskID);
-	}
-
-	/**
-	 * Returns the {@link SubtaskMetricStore} for the given job/task ID and subtask index.
-	 *
-	 * @param jobID        job ID
-	 * @param taskID       task ID
-	 * @param subtaskIndex subtask index
-	 * @return SubtaskMetricStore for the given IDs and index, or null if no store for the given arguments exists
-	 */
-	public SubtaskMetricStore getSubtaskMetricStore(String jobID, String taskID, int subtaskIndex) {
-		TaskMetricStore task = getTaskMetricStore(jobID, taskID);
-		if (task == null) {
-			return null;
-		}
-		return task.getSubtaskMetricStore(subtaskIndex);
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// sub MetricStore classes
-	// -----------------------------------------------------------------------------------------------------------------
-	private abstract static class ComponentMetricStore {
-		public final Map<String, String> metrics = new HashMap<>();
-
-		public String getMetric(String name, String defaultValue) {
-			String value = this.metrics.get(name);
-			return value != null
-				? value
-				: defaultValue;
-		}
-	}
-
-	/**
-	 * Sub-structure containing metrics of the JobManager.
-	 */
-	public static class JobManagerMetricStore extends ComponentMetricStore {
-	}
-
-	/**
-	 * Sub-structure containing metrics of a single TaskManager.
-	 */
-	public static class TaskManagerMetricStore extends ComponentMetricStore {
-		public final Set<String> garbageCollectorNames = new HashSet<>();
-
-		public void addGarbageCollectorName(String name) {
-			garbageCollectorNames.add(name);
-		}
-	}
-
-	/**
-	 * Sub-structure containing metrics of a single Job.
-	 */
-	public static class JobMetricStore extends ComponentMetricStore {
-		private final Map<String, TaskMetricStore> tasks = new HashMap<>();
-
-		public TaskMetricStore getTaskMetricStore(String taskID) {
-			return tasks.get(taskID);
-		}
-	}
-
-	/**
-	 * Sub-structure containing metrics of a single Task.
-	 */
-	public static class TaskMetricStore extends ComponentMetricStore {
-		private final Map<Integer, SubtaskMetricStore> subtasks = new HashMap<>();
-
-		public SubtaskMetricStore getSubtaskMetricStore(int subtaskIndex) {
-			return subtasks.get(subtaskIndex);
-		}
-	}
-
-	/**
-	 * Sub-structure containing metrics of a single Subtask.
-	 */
-	public static class SubtaskMetricStore extends ComponentMetricStore {
-	}
-}


[15/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
deleted file mode 100644
index 1ec3f9c..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
+++ /dev/null
@@ -1,87 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
-
-import java.nio.charset.Charset;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for most request handlers. The handlers must produce a JSON response.
- */
-public abstract class AbstractJsonRequestHandler implements RequestHandler {
-
-	private static final Charset ENCODING = Charset.forName("UTF-8");
-
-	protected final Executor executor;
-
-	protected AbstractJsonRequestHandler(Executor executor) {
-		this.executor = Preconditions.checkNotNull(executor);
-	}
-
-	@Override
-	public CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		CompletableFuture<String> resultFuture = handleJsonRequest(pathParams, queryParams, jobManagerGateway);
-
-		return resultFuture.thenApplyAsync(
-			(String result) -> {
-				byte[] bytes = result.getBytes(ENCODING);
-
-				DefaultFullHttpResponse response = new DefaultFullHttpResponse(
-					HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes));
-
-				response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
-				response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
-
-				return response;
-			});
-	}
-
-	/**
-	 * Core method that handles the request and generates the response. The method needs to
-	 * respond with a valid JSON string. Exceptions may be thrown and will be handled.
-	 *
-	 * @param pathParams The map of REST path parameters, decoded by the router.
-	 * @param queryParams The map of query parameters.
-	 * @param jobManagerGateway to communicate with the JobManager.
-	 *
-	 * @return The JSON string that is the HTTP response.
-	 *
-	 * @throws Exception Handlers may forward exceptions. Exceptions of type
-	 *         {@link org.apache.flink.runtime.webmonitor.NotFoundException} will cause a HTTP 404
-	 *         response with the exception message, other exceptions will cause a HTTP 500 response
-	 *         with the exception stack trace.
-	 */
-	public abstract CompletableFuture<String> handleJsonRequest(
-			Map<String, String> pathParams,
-			Map<String, String> queryParams,
-			JobManagerGateway jobManagerGateway);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java
deleted file mode 100644
index 1b20673..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java
+++ /dev/null
@@ -1,78 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.util.FlinkException;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for request handlers whose response depends on a specific subtask execution attempt
- * (defined via the "attempt" parameter) of a specific subtask (defined via the
- * "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
- * specific job, defined via (defined voa the "jobid" parameter).
- */
-public abstract class AbstractSubtaskAttemptRequestHandler extends AbstractSubtaskRequestHandler {
-
-	public AbstractSubtaskAttemptRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params) {
-		final String attemptNumberString = params.get("attempt");
-		if (attemptNumberString == null) {
-			return FutureUtils.completedExceptionally(new FlinkException("Attempt number parameter missing"));
-		}
-
-		final int attempt;
-		try {
-			attempt = Integer.parseInt(attemptNumberString);
-		}
-		catch (NumberFormatException e) {
-			return FutureUtils.completedExceptionally(new FlinkException("Invalid attempt number parameter"));
-		}
-
-		final AccessExecution currentAttempt = vertex.getCurrentExecutionAttempt();
-		if (attempt == currentAttempt.getAttemptNumber()) {
-			return handleRequest(currentAttempt, params);
-		}
-		else if (attempt >= 0 && attempt < currentAttempt.getAttemptNumber()) {
-			AccessExecution exec = vertex.getPriorExecutionAttempt(attempt);
-
-			if (exec != null) {
-				return handleRequest(exec, params);
-			} else {
-				return FutureUtils.completedExceptionally(new RequestHandlerException("Execution for attempt " + attempt +
-					" has already been deleted."));
-			}
-		}
-		else {
-			return FutureUtils.completedExceptionally(new FlinkException("Attempt does not exist: " + attempt));
-		}
-	}
-
-	public abstract CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java
deleted file mode 100644
index ab85034..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java
+++ /dev/null
@@ -1,66 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.util.FlinkException;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Base class for request handlers whose response depends on a specific subtask (defined via the
- * "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
- * specific job, defined via (defined voa the "jobid" parameter).
- */
-public abstract class AbstractSubtaskRequestHandler extends AbstractJobVertexRequestHandler {
-
-	public AbstractSubtaskRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public final CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
-		final String subtaskNumberString = params.get("subtasknum");
-		if (subtaskNumberString == null) {
-			return FutureUtils.completedExceptionally(new FlinkException("Subtask number parameter missing"));
-		}
-
-		final int subtask;
-		try {
-			subtask = Integer.parseInt(subtaskNumberString);
-		}
-		catch (NumberFormatException e) {
-			return FutureUtils.completedExceptionally(new FlinkException("Invalid subtask number parameter", e));
-		}
-
-		if (subtask < 0 || subtask >= jobVertex.getParallelism()) {
-			return FutureUtils.completedExceptionally(new FlinkException("subtask does not exist: " + subtask));
-		}
-
-		final AccessExecutionVertex vertex = jobVertex.getTaskVertices()[subtask];
-		return handleRequest(vertex, params);
-	}
-
-	public abstract CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
deleted file mode 100644
index 17db2e8..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
+++ /dev/null
@@ -1,105 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.webmonitor.StatusOverview;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.util.FlinkException;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Responder that returns the status of the Flink cluster, such as how many
- * TaskManagers are currently connected, and how many jobs are running.
- */
-public class ClusterOverviewHandler extends AbstractJsonRequestHandler {
-
-	private static final String CLUSTER_OVERVIEW_REST_PATH = "/overview";
-
-	private static final String version = EnvironmentInformation.getVersion();
-
-	private static final String commitID = EnvironmentInformation.getRevisionInformation().commitId;
-
-	private final Time timeout;
-
-	public ClusterOverviewHandler(Executor executor, Time timeout) {
-		super(executor);
-		this.timeout = checkNotNull(timeout);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CLUSTER_OVERVIEW_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		// we need no parameters, get all requests
-		try {
-			if (jobManagerGateway != null) {
-				CompletableFuture<StatusOverview> overviewFuture = jobManagerGateway.requestStatusOverview(timeout);
-
-				return overviewFuture.thenApplyAsync(
-					(StatusOverview overview) -> {
-						StringWriter writer = new StringWriter();
-						try {
-							JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-							gen.writeStartObject();
-							gen.writeNumberField("taskmanagers", overview.getNumTaskManagersConnected());
-							gen.writeNumberField("slots-total", overview.getNumSlotsTotal());
-							gen.writeNumberField("slots-available", overview.getNumSlotsAvailable());
-							gen.writeNumberField("jobs-running", overview.getNumJobsRunningOrPending());
-							gen.writeNumberField("jobs-finished", overview.getNumJobsFinished());
-							gen.writeNumberField("jobs-cancelled", overview.getNumJobsCancelled());
-							gen.writeNumberField("jobs-failed", overview.getNumJobsFailed());
-							gen.writeStringField("flink-version", version);
-							if (!commitID.equals(EnvironmentInformation.UNKNOWN)) {
-								gen.writeStringField("flink-commit", commitID);
-							}
-							gen.writeEndObject();
-
-							gen.close();
-							return writer.toString();
-						} catch (IOException exception) {
-							throw new FlinkFutureException("Could not write cluster overview.", exception);
-						}
-					},
-					executor);
-			} else {
-				throw new Exception("No connection to the leading JobManager.");
-			}
-		}
-		catch (Exception e) {
-			return FutureUtils.completedExceptionally(new FlinkException("Failed to fetch list of all running jobs: ", e));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
deleted file mode 100644
index 34898e7..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
+++ /dev/null
@@ -1,57 +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.webmonitor.handlers;
-
-import org.apache.flink.configuration.ConfigConstants;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
-
-/**
- * Responder that returns a constant String.
- */
-@ChannelHandler.Sharable
-public class ConstantTextHandler extends SimpleChannelInboundHandler<Routed> {
-
-	private final byte[] encodedText;
-
-	public ConstantTextHandler(String text) {
-		this.encodedText = text.getBytes(ConfigConstants.DEFAULT_CHARSET);
-	}
-
-	@Override
-	protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
-		HttpResponse response = new DefaultFullHttpResponse(
-			HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(encodedText));
-
-		response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, encodedText.length);
-		response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain");
-
-		KeepAliveWrite.flush(ctx, routed.request(), response);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
deleted file mode 100644
index acf1cd0..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
+++ /dev/null
@@ -1,112 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.TimeUnit;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Responder that returns with a list of all JobIDs of jobs found at the target actor.
- * May serve the IDs of current jobs, or past jobs, depending on whether this handler is
- * given the JobManager or Archive Actor Reference.
- */
-public class CurrentJobIdsHandler extends AbstractJsonRequestHandler {
-
-	private static final String CURRENT_JOB_IDS_REST_PATH = "/jobs";
-
-	private final Time timeout;
-
-	public CurrentJobIdsHandler(Executor executor, Time timeout) {
-		super(executor);
-		this.timeout = requireNonNull(timeout);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{CURRENT_JOB_IDS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				// we need no parameters, get all requests
-				try {
-					if (jobManagerGateway != null) {
-						CompletableFuture<JobsWithIDsOverview> overviewFuture = jobManagerGateway.requestJobsOverview(timeout);
-						JobsWithIDsOverview overview = overviewFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS);
-
-						StringWriter writer = new StringWriter();
-						JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-						gen.writeStartObject();
-
-						gen.writeArrayFieldStart("jobs-running");
-						for (JobID jid : overview.getJobsRunningOrPending()) {
-							gen.writeString(jid.toString());
-						}
-						gen.writeEndArray();
-
-						gen.writeArrayFieldStart("jobs-finished");
-						for (JobID jid : overview.getJobsFinished()) {
-							gen.writeString(jid.toString());
-						}
-						gen.writeEndArray();
-
-						gen.writeArrayFieldStart("jobs-cancelled");
-						for (JobID jid : overview.getJobsCancelled()) {
-							gen.writeString(jid.toString());
-						}
-						gen.writeEndArray();
-
-						gen.writeArrayFieldStart("jobs-failed");
-						for (JobID jid : overview.getJobsFailed()) {
-							gen.writeString(jid.toString());
-						}
-						gen.writeEndArray();
-
-						gen.writeEndObject();
-
-						gen.close();
-						return writer.toString();
-					}
-					else {
-						throw new Exception("No connection to the leading JobManager.");
-					}
-				}
-				catch (Exception e) {
-					throw new FlinkFutureException("Failed to fetch list of all running jobs.", e);
-				}
-			},
-			executor);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
deleted file mode 100644
index a5b116c..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
+++ /dev/null
@@ -1,182 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.webmonitor.JobDetails;
-import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
-import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Request handler that returns a summary of the job status.
- */
-public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
-
-	private static final String ALL_JOBS_REST_PATH = "/joboverview";
-	private static final String RUNNING_JOBS_REST_PATH = "/joboverview/running";
-	private static final String COMPLETED_JOBS_REST_PATH = "/joboverview/completed";
-
-	private final Time timeout;
-
-	private final boolean includeRunningJobs;
-	private final boolean includeFinishedJobs;
-
-	public CurrentJobsOverviewHandler(
-			Executor executor,
-			Time timeout,
-			boolean includeRunningJobs,
-			boolean includeFinishedJobs) {
-
-		super(executor);
-		this.timeout = checkNotNull(timeout);
-		this.includeRunningJobs = includeRunningJobs;
-		this.includeFinishedJobs = includeFinishedJobs;
-	}
-
-	@Override
-	public String[] getPaths() {
-		if (includeRunningJobs && includeFinishedJobs) {
-			return new String[]{ALL_JOBS_REST_PATH};
-		}
-		if (includeRunningJobs) {
-			return new String[]{RUNNING_JOBS_REST_PATH};
-		} else {
-			return new String[]{COMPLETED_JOBS_REST_PATH};
-		}
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		if (jobManagerGateway != null) {
-			CompletableFuture<MultipleJobsDetails> jobDetailsFuture = jobManagerGateway.requestJobDetails(includeRunningJobs, includeFinishedJobs, timeout);
-
-			return jobDetailsFuture.thenApplyAsync(
-				(MultipleJobsDetails result) -> {
-					final long now = System.currentTimeMillis();
-
-					StringWriter writer = new StringWriter();
-					try {
-						JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-						gen.writeStartObject();
-
-						if (includeRunningJobs && includeFinishedJobs) {
-							gen.writeArrayFieldStart("running");
-							for (JobDetails detail : result.getRunningJobs()) {
-								writeJobDetailOverviewAsJson(detail, gen, now);
-							}
-							gen.writeEndArray();
-
-							gen.writeArrayFieldStart("finished");
-							for (JobDetails detail : result.getFinishedJobs()) {
-								writeJobDetailOverviewAsJson(detail, gen, now);
-							}
-							gen.writeEndArray();
-						} else {
-							gen.writeArrayFieldStart("jobs");
-							for (JobDetails detail : includeRunningJobs ? result.getRunningJobs() : result.getFinishedJobs()) {
-								writeJobDetailOverviewAsJson(detail, gen, now);
-							}
-							gen.writeEndArray();
-						}
-
-						gen.writeEndObject();
-						gen.close();
-						return writer.toString();
-					} catch (IOException e) {
-						throw new FlinkFutureException("Could not write current jobs overview json.", e);
-					}
-				},
-				executor);
-		}
-		else {
-			return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
-		}
-	}
-
-	/**
-	 * Archivist for the CurrentJobsOverviewHandler.
-	 */
-	public static class CurrentJobsOverviewJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			StringWriter writer = new StringWriter();
-			try (JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
-				gen.writeStartObject();
-				gen.writeArrayFieldStart("running");
-				gen.writeEndArray();
-				gen.writeArrayFieldStart("finished");
-				writeJobDetailOverviewAsJson(WebMonitorUtils.createDetailsForJob(graph), gen, System.currentTimeMillis());
-				gen.writeEndArray();
-				gen.writeEndObject();
-			}
-			String json = writer.toString();
-			String path = ALL_JOBS_REST_PATH;
-			return Collections.singleton(new ArchivedJson(path, json));
-		}
-	}
-
-	public static void writeJobDetailOverviewAsJson(JobDetails details, JsonGenerator gen, long now) throws IOException {
-		gen.writeStartObject();
-
-		gen.writeStringField("jid", details.getJobId().toString());
-		gen.writeStringField("name", details.getJobName());
-		gen.writeStringField("state", details.getStatus().name());
-
-		gen.writeNumberField("start-time", details.getStartTime());
-		gen.writeNumberField("end-time", details.getEndTime());
-		gen.writeNumberField("duration", (details.getEndTime() <= 0 ? now : details.getEndTime()) - details.getStartTime());
-		gen.writeNumberField("last-modification", details.getLastUpdateTime());
-
-		gen.writeObjectFieldStart("tasks");
-		gen.writeNumberField("total", details.getNumTasks());
-
-		final int[] perState = details.getNumVerticesPerExecutionState();
-		gen.writeNumberField("pending", perState[ExecutionState.CREATED.ordinal()] +
-				perState[ExecutionState.SCHEDULED.ordinal()] +
-				perState[ExecutionState.DEPLOYING.ordinal()]);
-		gen.writeNumberField("running", perState[ExecutionState.RUNNING.ordinal()]);
-		gen.writeNumberField("finished", perState[ExecutionState.FINISHED.ordinal()]);
-		gen.writeNumberField("canceling", perState[ExecutionState.CANCELING.ordinal()]);
-		gen.writeNumberField("canceled", perState[ExecutionState.CANCELED.ordinal()]);
-		gen.writeNumberField("failed", perState[ExecutionState.FAILED.ordinal()]);
-		gen.writeEndObject();
-
-		gen.writeEndObject();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
deleted file mode 100644
index 39984b1..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
+++ /dev/null
@@ -1,90 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Responder that returns the parameters that define how the asynchronous requests
- * against this web server should behave. It defines for example the refresh interval,
- * and time zone of the server timestamps.
- */
-public class DashboardConfigHandler extends AbstractJsonRequestHandler {
-
-	private static final String DASHBOARD_CONFIG_REST_PATH = "/config";
-
-	private final String configString;
-
-	public DashboardConfigHandler(Executor executor, long refreshInterval) {
-		super(executor);
-		try {
-			this.configString = createConfigJson(refreshInterval);
-		}
-		catch (Exception e) {
-			// should never happen
-			throw new RuntimeException(e.getMessage(), e);
-		}
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{DASHBOARD_CONFIG_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.completedFuture(configString);
-	}
-
-	public static String createConfigJson(long refreshInterval) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		TimeZone timeZone = TimeZone.getDefault();
-		String timeZoneName = timeZone.getDisplayName();
-		long timeZoneOffset = timeZone.getRawOffset();
-
-		gen.writeStartObject();
-		gen.writeNumberField("refresh-interval", refreshInterval);
-		gen.writeNumberField("timezone-offset", timeZoneOffset);
-		gen.writeStringField("timezone-name", timeZoneName);
-		gen.writeStringField("flink-version", EnvironmentInformation.getVersion());
-
-		EnvironmentInformation.RevisionInformation revision = EnvironmentInformation.getRevisionInformation();
-		if (revision != null) {
-			gen.writeStringField("flink-revision", revision.commitId + " @ " + revision.commitDate);
-		}
-
-		gen.writeEndObject();
-
-		gen.close();
-
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java
index 978432b..c95cc32 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
 
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
index 0b0d32e..760c836 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
@@ -35,6 +35,8 @@ import org.apache.flink.optimizer.plan.StreamingPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
 import org.apache.flink.util.ExceptionUtils;
 
 import com.fasterxml.jackson.core.JsonGenerator;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java
index d9df1d4..04f663d 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java
@@ -20,6 +20,8 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.concurrent.FlinkFutureException;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
 
 import com.fasterxml.jackson.core.JsonGenerator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java
index 95281a4..4248dd4 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java
@@ -21,6 +21,8 @@ package org.apache.flink.runtime.webmonitor.handlers;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.runtime.concurrent.FlinkFutureException;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
 import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
 
 import com.fasterxml.jackson.core.JsonGenerator;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java
index b117b3d..4d79492 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java
@@ -22,6 +22,7 @@ import org.apache.flink.runtime.concurrent.FlinkFutureException;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
 
 import com.fasterxml.jackson.core.JsonGenerator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
index 7ada0b4..16a1565 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.concurrent.FlinkFutureException;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.JsonFactory;
 import org.apache.flink.util.Preconditions;
 
 import com.fasterxml.jackson.core.JsonGenerator;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java
index 61b3f58..9a0bac4 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler;
 
 import java.io.File;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java
deleted file mode 100644
index 4dede3a..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java
+++ /dev/null
@@ -1,107 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the aggregated user accumulators of a job.
- */
-public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String JOB_ACCUMULATORS_REST_PATH = "/jobs/:jobid/accumulators";
-
-	public JobAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_ACCUMULATORS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createJobAccumulatorsJson(graph);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create job accumulators json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the JobAccumulatorsHandler.
-	 */
-	public static class JobAccumulatorsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createJobAccumulatorsJson(graph);
-			String path = JOB_ACCUMULATORS_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-
-	public static String createJobAccumulatorsJson(AccessExecutionGraph graph) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		StringifiedAccumulatorResult[] allAccumulators = graph.getAccumulatorResultsStringified();
-
-		gen.writeStartObject();
-
-		gen.writeArrayFieldStart("job-accumulators");
-		// empty for now
-		gen.writeEndArray();
-
-		gen.writeArrayFieldStart("user-task-accumulators");
-		for (StringifiedAccumulatorResult acc : allAccumulators) {
-			gen.writeStartObject();
-			gen.writeStringField("name", acc.getName());
-			gen.writeStringField("type", acc.getType());
-			gen.writeStringField("value", acc.getValue());
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java
deleted file mode 100644
index 1a7d868..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java
+++ /dev/null
@@ -1,72 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.StringUtils;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler for the CANCEL request.
- */
-public class JobCancellationHandler extends AbstractJsonRequestHandler {
-
-	private static final String JOB_CONCELLATION_REST_PATH = "/jobs/:jobid/cancel";
-	private static final String JOB_CONCELLATION_YARN_REST_PATH = "/jobs/:jobid/yarn-cancel";
-
-	private final Time timeout;
-
-	public JobCancellationHandler(Executor executor, Time timeout) {
-		super(executor);
-		this.timeout = Preconditions.checkNotNull(timeout);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_CONCELLATION_REST_PATH, JOB_CONCELLATION_YARN_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid")));
-					if (jobManagerGateway != null) {
-						jobManagerGateway.cancelJob(jobId, timeout);
-						return "{}";
-					}
-					else {
-						throw new Exception("No connection to the leading JobManager.");
-					}
-				}
-				catch (Exception e) {
-					throw new FlinkFutureException("Failed to cancel the job with id: "  + pathParams.get("jobid"), e);
-				}
-			},
-			executor);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
deleted file mode 100644
index 4e41447..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
+++ /dev/null
@@ -1,428 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoint;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.NotFoundException;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.Charset;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Request handler for {@link CancelJobWithSavepoint} messages.
- */
-public class JobCancellationWithSavepointHandlers {
-
-	private static final String CANCEL_WITH_SAVEPOINT_REST_PATH = "/jobs/:jobid/cancel-with-savepoint";
-	private static final String CANCEL_WITH_SAVEPOINT_DIRECTORY_REST_PATH = "/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory";
-
-	/** URL for in-progress cancellations. */
-	private static final String CANCELLATION_IN_PROGRESS_REST_PATH = "/jobs/:jobid/cancel-with-savepoint/in-progress/:requestId";
-
-	/** Encodings for String. */
-	private static final Charset ENCODING = ConfigConstants.DEFAULT_CHARSET;
-
-	/** Shared lock between Trigger and In-Progress handlers. */
-	private final Object lock = new Object();
-
-	/** In-Progress requests. */
-	private final Map<JobID, Long> inProgress = new HashMap<>();
-
-	/** Succeeded/failed request. Either String or Throwable. */
-	private final Map<Long, Object> completed = new HashMap<>();
-
-	/** Atomic request counter. */
-	private long requestCounter;
-
-	/** Handler for trigger requests. */
-	private final TriggerHandler triggerHandler;
-
-	/** Handler for in-progress requests. */
-	private final InProgressHandler inProgressHandler;
-
-	/** Default savepoint directory. */
-	private final String defaultSavepointDirectory;
-
-	public JobCancellationWithSavepointHandlers(
-			ExecutionGraphHolder currentGraphs,
-			Executor executor) {
-		this(currentGraphs, executor, null);
-	}
-
-	public JobCancellationWithSavepointHandlers(
-			ExecutionGraphHolder currentGraphs,
-			Executor executor,
-			@Nullable String defaultSavepointDirectory) {
-
-		this.triggerHandler = new TriggerHandler(currentGraphs, executor);
-		this.inProgressHandler = new InProgressHandler();
-		this.defaultSavepointDirectory = defaultSavepointDirectory;
-	}
-
-	public TriggerHandler getTriggerHandler() {
-		return triggerHandler;
-	}
-
-	public InProgressHandler getInProgressHandler() {
-		return inProgressHandler;
-	}
-
-	// ------------------------------------------------------------------------
-	// New requests
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Handler for triggering a {@link CancelJobWithSavepoint} message.
-	 */
-	class TriggerHandler implements RequestHandler {
-
-		/** Current execution graphs. */
-		private final ExecutionGraphHolder currentGraphs;
-
-		/** Execution context for futures. */
-		private final Executor executor;
-
-		public TriggerHandler(ExecutionGraphHolder currentGraphs, Executor executor) {
-			this.currentGraphs = checkNotNull(currentGraphs);
-			this.executor = checkNotNull(executor);
-		}
-
-		@Override
-		public String[] getPaths() {
-			return new String[]{CANCEL_WITH_SAVEPOINT_REST_PATH, CANCEL_WITH_SAVEPOINT_DIRECTORY_REST_PATH};
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public CompletableFuture<FullHttpResponse> handleRequest(
-				Map<String, String> pathParams,
-				Map<String, String> queryParams,
-				JobManagerGateway jobManagerGateway) {
-
-			if (jobManagerGateway != null) {
-				JobID jobId = JobID.fromHexString(pathParams.get("jobid"));
-				final CompletableFuture<Optional<AccessExecutionGraph>> graphFuture;
-
-				graphFuture = currentGraphs.getExecutionGraph(jobId, jobManagerGateway);
-
-				return graphFuture.thenApplyAsync(
-					(Optional<AccessExecutionGraph> optGraph) -> {
-						final AccessExecutionGraph graph = optGraph.orElseThrow(
-							() -> new FlinkFutureException(
-								new NotFoundException("Could not find ExecutionGraph with jobId " + jobId + '.')));
-
-						CheckpointCoordinator coord = graph.getCheckpointCoordinator();
-						if (coord == null) {
-							throw new FlinkFutureException(new Exception("Cannot find CheckpointCoordinator for job."));
-						}
-
-						String targetDirectory = pathParams.get("targetDirectory");
-						if (targetDirectory == null) {
-							if (defaultSavepointDirectory == null) {
-								throw new IllegalStateException("No savepoint directory configured. " +
-									"You can either specify a directory when triggering this savepoint or " +
-									"configure a cluster-wide default via key '" +
-									CoreOptions.SAVEPOINT_DIRECTORY.key() + "'.");
-							} else {
-								targetDirectory = defaultSavepointDirectory;
-							}
-						}
-
-						try {
-							return handleNewRequest(jobManagerGateway, jobId, targetDirectory, coord.getCheckpointTimeout());
-						} catch (IOException e) {
-							throw new FlinkFutureException("Could not cancel job with savepoint.", e);
-						}
-					}, executor);
-			} else {
-				return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager."));
-			}
-		}
-
-		@SuppressWarnings("unchecked")
-		private FullHttpResponse handleNewRequest(JobManagerGateway jobManagerGateway, final JobID jobId, String targetDirectory, long checkpointTimeout) throws IOException {
-			// Check whether a request exists
-			final long requestId;
-			final boolean isNewRequest;
-			synchronized (lock) {
-				if (inProgress.containsKey(jobId)) {
-					requestId = inProgress.get(jobId);
-					isNewRequest = false;
-				} else {
-					requestId = ++requestCounter;
-					inProgress.put(jobId, requestId);
-					isNewRequest = true;
-				}
-			}
-
-			if (isNewRequest) {
-				boolean success = false;
-
-				try {
-					// Trigger cancellation
-					CompletableFuture<String> cancelJobFuture = jobManagerGateway
-						.cancelJobWithSavepoint(jobId, targetDirectory, Time.milliseconds(checkpointTimeout));
-
-					cancelJobFuture.whenCompleteAsync(
-						(String path, Throwable throwable) -> {
-							try {
-								if (throwable != null) {
-									completed.put(requestId, throwable);
-								} else {
-									completed.put(requestId, path);
-								}
-							} finally {
-								inProgress.remove(jobId);
-							}
-						}, executor);
-
-					success = true;
-				} finally {
-					synchronized (lock) {
-						if (!success) {
-							inProgress.remove(jobId);
-						}
-					}
-				}
-			}
-
-			// In-progress location
-			String location = CANCELLATION_IN_PROGRESS_REST_PATH
-					.replace(":jobid", jobId.toString())
-					.replace(":requestId", Long.toString(requestId));
-
-			// Accepted response
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-			gen.writeStartObject();
-			gen.writeStringField("status", "accepted");
-			gen.writeNumberField("request-id", requestId);
-			gen.writeStringField("location", location);
-			gen.writeEndObject();
-			gen.close();
-
-			String json = writer.toString();
-			byte[] bytes = json.getBytes(ENCODING);
-
-			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
-					HttpVersion.HTTP_1_1,
-					HttpResponseStatus.ACCEPTED,
-					Unpooled.wrappedBuffer(bytes));
-
-			response.headers().set(HttpHeaders.Names.LOCATION, location);
-
-			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
-			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
-
-			FullHttpResponse accepted = response;
-
-			return accepted;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// In-progress requests
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Handler for in-progress cancel with savepoint operations.
-	 */
-	class InProgressHandler implements RequestHandler {
-
-		/** The number of recent checkpoints whose IDs are remembered. */
-		private static final int NUM_GHOST_REQUEST_IDS = 16;
-
-		/** Remember some recently completed. */
-		private final ArrayDeque<Tuple2<Long, Object>> recentlyCompleted = new ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
-
-		@Override
-		public String[] getPaths() {
-			return new String[]{CANCELLATION_IN_PROGRESS_REST_PATH};
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
-			JobID jobId = JobID.fromHexString(pathParams.get("jobid"));
-			long requestId = Long.parseLong(pathParams.get("requestId"));
-
-			return CompletableFuture.supplyAsync(
-				() -> {
-					try {
-						synchronized (lock) {
-							Object result = completed.remove(requestId);
-
-							if (result != null) {
-								// Add to recent history
-								recentlyCompleted.add(new Tuple2<>(requestId, result));
-								if (recentlyCompleted.size() > NUM_GHOST_REQUEST_IDS) {
-									recentlyCompleted.remove();
-								}
-
-								if (result.getClass() == String.class) {
-									String savepointPath = (String) result;
-									return createSuccessResponse(requestId, savepointPath);
-								} else {
-									Throwable cause = (Throwable) result;
-									return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage());
-								}
-							} else {
-								// Check in-progress
-								Long inProgressRequestId = inProgress.get(jobId);
-								if (inProgressRequestId != null) {
-									// Sanity check
-									if (inProgressRequestId == requestId) {
-										return createInProgressResponse(requestId);
-									} else {
-										String msg = "Request ID does not belong to JobID";
-										return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, msg);
-									}
-								}
-
-								// Check recent history
-								for (Tuple2<Long, Object> recent : recentlyCompleted) {
-									if (recent.f0 == requestId) {
-										if (recent.f1.getClass() == String.class) {
-											String savepointPath = (String) recent.f1;
-											return createSuccessResponse(requestId, savepointPath);
-										} else {
-											Throwable cause = (Throwable) recent.f1;
-											return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage());
-										}
-									}
-								}
-
-								return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, "Unknown job/request ID");
-							}
-						}
-					} catch (Exception e) {
-						throw new FlinkFutureException("Could not handle in progress request.", e);
-					}
-				});
-		}
-
-		private FullHttpResponse createSuccessResponse(long requestId, String savepointPath) throws IOException {
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-			gen.writeStartObject();
-
-			gen.writeStringField("status", "success");
-			gen.writeNumberField("request-id", requestId);
-			gen.writeStringField("savepoint-path", savepointPath);
-
-			gen.writeEndObject();
-			gen.close();
-
-			String json = writer.toString();
-			byte[] bytes = json.getBytes(ENCODING);
-
-			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
-					HttpVersion.HTTP_1_1,
-					HttpResponseStatus.CREATED,
-					Unpooled.wrappedBuffer(bytes));
-
-			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
-			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
-
-			return response;
-		}
-
-		private FullHttpResponse createInProgressResponse(long requestId) throws IOException {
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-			gen.writeStartObject();
-
-			gen.writeStringField("status", "in-progress");
-			gen.writeNumberField("request-id", requestId);
-
-			gen.writeEndObject();
-			gen.close();
-
-			String json = writer.toString();
-			byte[] bytes = json.getBytes(ENCODING);
-
-			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
-					HttpVersion.HTTP_1_1,
-					HttpResponseStatus.ACCEPTED,
-					Unpooled.wrappedBuffer(bytes));
-
-			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
-			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
-
-			return response;
-		}
-
-		private FullHttpResponse createFailureResponse(HttpResponseStatus code, long requestId, String errMsg) throws IOException {
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-			gen.writeStartObject();
-
-			gen.writeStringField("status", "failed");
-			gen.writeNumberField("request-id", requestId);
-			gen.writeStringField("cause", errMsg);
-
-			gen.writeEndObject();
-			gen.close();
-
-			String json = writer.toString();
-			byte[] bytes = json.getBytes(ENCODING);
-
-			DefaultFullHttpResponse response = new DefaultFullHttpResponse(
-					HttpVersion.HTTP_1_1,
-					code,
-					Unpooled.wrappedBuffer(bytes));
-
-			response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
-			response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
-
-			return response;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java
deleted file mode 100644
index 0b15b37..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java
+++ /dev/null
@@ -1,119 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.ArchivedExecutionConfig;
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the execution config of a job.
- */
-public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String JOB_CONFIG_REST_PATH = "/jobs/:jobid/config";
-
-	public JobConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_CONFIG_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createJobConfigJson(graph);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not write job config json.", e);
-				}
-			},
-			executor);
-
-	}
-
-	/**
-	 * Archivist for the JobConfigHandler.
-	 */
-	public static class JobConfigJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createJobConfigJson(graph);
-			String path = JOB_CONFIG_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-
-	public static String createJobConfigJson(AccessExecutionGraph graph) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-		gen.writeStringField("jid", graph.getJobID().toString());
-		gen.writeStringField("name", graph.getJobName());
-
-		final ArchivedExecutionConfig summary = graph.getArchivedExecutionConfig();
-
-		if (summary != null) {
-			gen.writeObjectFieldStart("execution-config");
-
-			gen.writeStringField("execution-mode", summary.getExecutionMode());
-
-			gen.writeStringField("restart-strategy", summary.getRestartStrategyDescription());
-			gen.writeNumberField("job-parallelism", summary.getParallelism());
-			gen.writeBooleanField("object-reuse-mode", summary.getObjectReuseEnabled());
-
-			Map<String, String> ucVals = summary.getGlobalJobParameters();
-			if (ucVals != null) {
-				gen.writeObjectFieldStart("user-config");
-
-				for (Map.Entry<String, String> ucVal : ucVals.entrySet()) {
-					gen.writeStringField(ucVal.getKey(), ucVal.getValue());
-				}
-
-				gen.writeEndObject();
-			}
-
-			gen.writeEndObject();
-		}
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java
deleted file mode 100644
index 8a50f87..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java
+++ /dev/null
@@ -1,225 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns details about a job. This includes:
- * <ul>
- *     <li>Dataflow plan</li>
- *     <li>id, name, and current status</li>
- *     <li>start time, end time, duration</li>
- *     <li>number of job vertices in each state (pending, running, finished, failed)</li>
- *     <li>info about job vertices, including runtime, status, I/O bytes and records, subtasks in each status</li>
- * </ul>
- */
-public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String JOB_DETAILS_REST_PATH = "/jobs/:jobid";
-	private static final String JOB_DETAILS_VERTICES_REST_PATH = "/jobs/:jobid/vertices";
-
-	private final MetricFetcher fetcher;
-
-	public JobDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
-		super(executionGraphHolder, executor);
-		this.fetcher = fetcher;
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_DETAILS_REST_PATH, JOB_DETAILS_VERTICES_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createJobDetailsJson(graph, fetcher);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create job details json.", e);
-				}
-			},
-			executor);
-	}
-
-	/**
-	 * Archivist for the JobDetailsHandler.
-	 */
-	public static class JobDetailsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createJobDetailsJson(graph, null);
-			String path1 = JOB_DETAILS_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			String path2 = JOB_DETAILS_VERTICES_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			Collection<ArchivedJson> archives = new ArrayList<>();
-			archives.add(new ArchivedJson(path1, json));
-			archives.add(new ArchivedJson(path2, json));
-			return archives;
-		}
-	}
-
-	public static String createJobDetailsJson(AccessExecutionGraph graph, @Nullable MetricFetcher fetcher) throws IOException {
-		final StringWriter writer = new StringWriter();
-		final JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		final long now = System.currentTimeMillis();
-
-		gen.writeStartObject();
-
-		// basic info
-		gen.writeStringField("jid", graph.getJobID().toString());
-		gen.writeStringField("name", graph.getJobName());
-		gen.writeBooleanField("isStoppable", graph.isStoppable());
-		gen.writeStringField("state", graph.getState().name());
-
-		// times and duration
-		final long jobStartTime = graph.getStatusTimestamp(JobStatus.CREATED);
-		final long jobEndTime = graph.getState().isGloballyTerminalState() ?
-				graph.getStatusTimestamp(graph.getState()) : -1L;
-		gen.writeNumberField("start-time", jobStartTime);
-		gen.writeNumberField("end-time", jobEndTime);
-		gen.writeNumberField("duration", (jobEndTime > 0 ? jobEndTime : now) - jobStartTime);
-		gen.writeNumberField("now", now);
-
-		// timestamps
-		gen.writeObjectFieldStart("timestamps");
-		for (JobStatus status : JobStatus.values()) {
-			gen.writeNumberField(status.name(), graph.getStatusTimestamp(status));
-		}
-		gen.writeEndObject();
-
-		// job vertices
-		int[] jobVerticesPerState = new int[ExecutionState.values().length];
-		gen.writeArrayFieldStart("vertices");
-
-		for (AccessExecutionJobVertex ejv : graph.getVerticesTopologically()) {
-			int[] tasksPerState = new int[ExecutionState.values().length];
-			long startTime = Long.MAX_VALUE;
-			long endTime = 0;
-			boolean allFinished = true;
-
-			for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
-				final ExecutionState state = vertex.getExecutionState();
-				tasksPerState[state.ordinal()]++;
-
-				// take the earliest start time
-				long started = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
-				if (started > 0) {
-					startTime = Math.min(startTime, started);
-				}
-
-				allFinished &= state.isTerminal();
-				endTime = Math.max(endTime, vertex.getStateTimestamp(state));
-			}
-
-			long duration;
-			if (startTime < Long.MAX_VALUE) {
-				if (allFinished) {
-					duration = endTime - startTime;
-				}
-				else {
-					endTime = -1L;
-					duration = now - startTime;
-				}
-			}
-			else {
-				startTime = -1L;
-				endTime = -1L;
-				duration = -1L;
-			}
-
-			ExecutionState jobVertexState =
-					ExecutionJobVertex.getAggregateJobVertexState(tasksPerState, ejv.getParallelism());
-			jobVerticesPerState[jobVertexState.ordinal()]++;
-
-			gen.writeStartObject();
-			gen.writeStringField("id", ejv.getJobVertexId().toString());
-			gen.writeStringField("name", ejv.getName());
-			gen.writeNumberField("parallelism", ejv.getParallelism());
-			gen.writeStringField("status", jobVertexState.name());
-
-			gen.writeNumberField("start-time", startTime);
-			gen.writeNumberField("end-time", endTime);
-			gen.writeNumberField("duration", duration);
-
-			gen.writeObjectFieldStart("tasks");
-			for (ExecutionState state : ExecutionState.values()) {
-				gen.writeNumberField(state.name(), tasksPerState[state.ordinal()]);
-			}
-			gen.writeEndObject();
-
-			MutableIOMetrics counts = new MutableIOMetrics();
-
-			for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
-				counts.addIOMetrics(
-					vertex.getCurrentExecutionAttempt(),
-					fetcher,
-					graph.getJobID().toString(),
-					ejv.getJobVertexId().toString());
-			}
-
-			counts.writeIOMetricsAsJson(gen);
-
-			gen.writeEndObject();
-		}
-		gen.writeEndArray();
-
-		gen.writeObjectFieldStart("status-counts");
-		for (ExecutionState state : ExecutionState.values()) {
-			gen.writeNumberField(state.name(), jobVerticesPerState[state.ordinal()]);
-		}
-		gen.writeEndObject();
-
-		gen.writeFieldName("plan");
-		gen.writeRawValue(graph.getJsonPlan());
-
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/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
deleted file mode 100644
index 6ffd443..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
+++ /dev/null
@@ -1,137 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.FlinkFutureException;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ErrorInfo;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.util.ExceptionUtils;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns the configuration of a job.
- */
-public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
-
-	private static final String JOB_EXCEPTIONS_REST_PATH = "/jobs/:jobid/exceptions";
-
-	static final int MAX_NUMBER_EXCEPTION_TO_REPORT = 20;
-
-	public JobExceptionsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
-		super(executionGraphHolder, executor);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{JOB_EXCEPTIONS_REST_PATH};
-	}
-
-	@Override
-	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
-		return CompletableFuture.supplyAsync(
-			() -> {
-				try {
-					return createJobExceptionsJson(graph);
-				} catch (IOException e) {
-					throw new FlinkFutureException("Could not create job exceptions json.", e);
-				}
-			},
-			executor
-		);
-	}
-
-	/**
-	 * Archivist for the JobExceptionsHandler.
-	 */
-	public static class JobExceptionsJsonArchivist implements JsonArchivist {
-
-		@Override
-		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
-			String json = createJobExceptionsJson(graph);
-			String path = JOB_EXCEPTIONS_REST_PATH
-				.replace(":jobid", graph.getJobID().toString());
-			return Collections.singletonList(new ArchivedJson(path, json));
-		}
-	}
-
-	public static String createJobExceptionsJson(AccessExecutionGraph graph) throws IOException {
-		StringWriter writer = new StringWriter();
-		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-		gen.writeStartObject();
-
-		// most important is the root failure cause
-		ErrorInfo rootException = graph.getFailureCause();
-		if (rootException != null && !rootException.getExceptionAsString().equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
-			gen.writeStringField("root-exception", rootException.getExceptionAsString());
-			gen.writeNumberField("timestamp", rootException.getTimestamp());
-		}
-
-		// we additionally collect all exceptions (up to a limit) that occurred in the individual tasks
-		gen.writeArrayFieldStart("all-exceptions");
-
-		int numExceptionsSoFar = 0;
-		boolean truncated = false;
-
-		for (AccessExecutionVertex task : graph.getAllExecutionVertices()) {
-			String t = task.getFailureCauseAsString();
-			if (t != null && !t.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
-				if (numExceptionsSoFar >= MAX_NUMBER_EXCEPTION_TO_REPORT) {
-					truncated = true;
-					break;
-				}
-
-				TaskManagerLocation location = task.getCurrentAssignedResourceLocation();
-				String locationString = location != null ?
-						location.getFQDNHostname() + ':' + location.dataPort() : "(unassigned)";
-
-				gen.writeStartObject();
-				gen.writeStringField("exception", t);
-				gen.writeStringField("task", task.getTaskNameWithSubtaskIndex());
-				gen.writeStringField("location", locationString);
-				long timestamp = task.getStateTimestamp(ExecutionState.FAILED);
-				gen.writeNumberField("timestamp", timestamp == 0 ? -1 : timestamp);
-				gen.writeEndObject();
-				numExceptionsSoFar++;
-			}
-		}
-		gen.writeEndArray();
-
-		gen.writeBooleanField("truncated", truncated);
-		gen.writeEndObject();
-
-		gen.close();
-		return writer.toString();
-	}
-}


[12/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
deleted file mode 100644
index fde16fc..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
+++ /dev/null
@@ -1,59 +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.webmonitor.metrics;
-
-import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
-
-import java.util.Map;
-import java.util.concurrent.Executor;
-
-/**
- * Request handler that returns for a given task manager a list of all available metrics or the values for a set of metrics.
- *
- * <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
- * {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
- *
- * <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
- * {@code /get?X,Y}
- * The handler will then return a list containing the values of the requested metrics.
- * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
- */
-public class TaskManagerMetricsHandler extends AbstractMetricsHandler {
-
-	private static final String TASKMANAGER_METRICS_REST_PATH = "/taskmanagers/:taskmanagerid/metrics";
-
-	public TaskManagerMetricsHandler(Executor executor, MetricFetcher fetcher) {
-		super(executor, fetcher);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{TASKMANAGER_METRICS_REST_PATH};
-	}
-
-	@Override
-	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
-		MetricStore.TaskManagerMetricStore taskManager = metrics.getTaskManagerMetricStore(pathParams.get(TaskManagersHandler.TASK_MANAGER_ID_KEY));
-		if (taskManager == null) {
-			return null;
-		} else {
-			return taskManager.metrics;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/MutableIOMetrics.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/MutableIOMetrics.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/MutableIOMetrics.java
deleted file mode 100644
index 9d71786..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/MutableIOMetrics.java
+++ /dev/null
@@ -1,109 +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.webmonitor.utils;
-
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.IOMetrics;
-import org.apache.flink.runtime.metrics.MetricNames;
-import org.apache.flink.runtime.webmonitor.handlers.JobVertexDetailsHandler;
-import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
-import org.apache.flink.runtime.webmonitor.metrics.MetricStore;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-
-/**
- * This class is a mutable version of the {@link IOMetrics} class that allows adding up IO-related metrics.
- *
- * <p>For finished jobs these metrics are stored in the {@link ExecutionGraph} as another {@link IOMetrics}.
- * For running jobs these metrics are retrieved using the {@link MetricFetcher}.
- *
- * <p>This class provides a common interface to handle both cases, reducing complexity in various handlers (like
- * the {@link JobVertexDetailsHandler}).
- */
-public class MutableIOMetrics extends IOMetrics {
-
-	private static final long serialVersionUID = -5460777634971381737L;
-
-	public MutableIOMetrics() {
-		super(0, 0, 0, 0, 0, 0.0D, 0.0D, 0.0D, 0.0D, 0.0D);
-	}
-
-	/**
-	 * Adds the IO metrics for the given attempt to this object. If the {@link AccessExecution} is in
-	 * a terminal state the contained {@link IOMetrics} object is added. Otherwise the given {@link MetricFetcher} is
-	 * used to retrieve the required metrics.
-	 *
-	 * @param attempt Attempt whose IO metrics should be added
-	 * @param fetcher MetricFetcher to retrieve metrics for running jobs
-	 * @param jobID JobID to which the attempt belongs
-	 * @param taskID TaskID to which the attempt belongs
-	 */
-	public void addIOMetrics(AccessExecution attempt, @Nullable MetricFetcher fetcher, String jobID, String taskID) {
-		if (attempt.getState().isTerminal()) {
-			IOMetrics ioMetrics = attempt.getIOMetrics();
-			if (ioMetrics != null) { // execAttempt is already finished, use final metrics stored in ExecutionGraph
-				this.numBytesInLocal += ioMetrics.getNumBytesInLocal();
-				this.numBytesInRemote += ioMetrics.getNumBytesInRemote();
-				this.numBytesOut += ioMetrics.getNumBytesOut();
-				this.numRecordsIn += ioMetrics.getNumRecordsIn();
-				this.numRecordsOut += ioMetrics.getNumRecordsOut();
-			}
-		} else { // execAttempt is still running, use MetricQueryService instead
-			if (fetcher != null) {
-				fetcher.update();
-				MetricStore.SubtaskMetricStore metrics = fetcher.getMetricStore().getSubtaskMetricStore(jobID, taskID, attempt.getParallelSubtaskIndex());
-				if (metrics != null) {
-					this.numBytesInLocal += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_IN_LOCAL, "0"));
-					this.numBytesInRemote += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_IN_REMOTE, "0"));
-					this.numBytesOut += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_BYTES_OUT, "0"));
-					this.numRecordsIn += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_RECORDS_IN, "0"));
-					this.numRecordsOut += Long.valueOf(metrics.getMetric(MetricNames.IO_NUM_RECORDS_OUT, "0"));
-				}
-			}
-		}
-	}
-
-	/**
-	 * Writes the IO metrics contained in this object to the given {@link JsonGenerator}.
-	 *
-	 * <p>The JSON structure written is as follows:
-	 * "metrics": {
-	 *     "read-bytes": 1,
-	 *     "write-bytes": 2,
-	 *     "read-records": 3,
-	 *     "write-records": 4
-	 * }
-	 *
-	 * @param gen JsonGenerator to which the metrics should be written
-	 * @throws IOException
-	 */
-	public void writeIOMetricsAsJson(JsonGenerator gen) throws IOException {
-		gen.writeObjectFieldStart("metrics");
-		gen.writeNumberField("read-bytes", this.numBytesInLocal + this.numBytesInRemote);
-		gen.writeNumberField("write-bytes", this.numBytesOut);
-		gen.writeNumberField("read-records", this.numRecordsIn);
-		gen.writeNumberField("write-records", this.numRecordsOut);
-		gen.writeEndObject();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
deleted file mode 100644
index 0e4734d..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
+++ /dev/null
@@ -1,332 +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.webmonitor;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.MemoryType;
-import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.client.JobClient;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import scala.Option;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Simple back pressured task test.
- */
-public class BackPressureStatsTrackerITCase extends TestLogger {
-
-	private static NetworkBufferPool networkBufferPool;
-	private static ActorSystem testActorSystem;
-
-	/** Shared as static variable with the test task. */
-	private static BufferPool testBufferPool;
-
-	@BeforeClass
-	public static void setup() {
-		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-		networkBufferPool = new NetworkBufferPool(100, 8192, MemoryType.HEAP);
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(testActorSystem);
-		networkBufferPool.destroy();
-	}
-
-	/**
-	 * Tests a simple fake-back pressured task. Back pressure is assumed when
-	 * sampled stack traces are in blocking buffer requests.
-	 */
-	@Test
-	public void testBackPressuredProducer() throws Exception {
-		new JavaTestKit(testActorSystem) {{
-			final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS);
-
-			// The JobGraph
-			final JobGraph jobGraph = new JobGraph();
-			final int parallelism = 4;
-
-			final JobVertex task = new JobVertex("Task");
-			task.setInvokableClass(BackPressuredTask.class);
-			task.setParallelism(parallelism);
-
-			jobGraph.addVertex(task);
-
-			final Configuration config = new Configuration();
-
-			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
-				config,
-				TestingUtils.defaultExecutor());
-
-			ActorGateway jobManger = null;
-			ActorGateway taskManager = null;
-
-			//
-			// 1) Consume all buffers at first (no buffers for the test task)
-			//
-			testBufferPool = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE);
-			final List<Buffer> buffers = new ArrayList<>();
-			while (true) {
-				Buffer buffer = testBufferPool.requestBuffer();
-				if (buffer != null) {
-					buffers.add(buffer);
-				} else {
-					break;
-				}
-			}
-
-			try {
-				jobManger = TestingUtils.createJobManager(
-					testActorSystem,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					highAvailabilityServices);
-
-				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
-
-				taskManager = TestingUtils.createTaskManager(
-					testActorSystem,
-					highAvailabilityServices,
-					config,
-					true,
-					true);
-
-				final ActorGateway jm = jobManger;
-
-				new Within(deadline) {
-					@Override
-					protected void run() {
-						try {
-							ActorGateway testActor = new AkkaActorGateway(getTestActor(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-
-							// Submit the job and wait until it is running
-							JobClient.submitJobDetached(
-									new AkkaJobManagerGateway(jm),
-									config,
-									jobGraph,
-									Time.milliseconds(deadline.toMillis()),
-									ClassLoader.getSystemClassLoader());
-
-							jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
-
-							expectMsgEquals(new AllVerticesRunning(jobGraph.getJobID()));
-
-							// Get the ExecutionGraph
-							jm.tell(new RequestExecutionGraph(jobGraph.getJobID()), testActor);
-
-							ExecutionGraphFound executionGraphResponse =
-									expectMsgClass(ExecutionGraphFound.class);
-
-							ExecutionGraph executionGraph = (ExecutionGraph) executionGraphResponse.executionGraph();
-							ExecutionJobVertex vertex = executionGraph.getJobVertex(task.getID());
-
-							StackTraceSampleCoordinator coordinator = new StackTraceSampleCoordinator(
-									testActorSystem.dispatcher(), 60000);
-
-							// Verify back pressure (clean up interval can be ignored)
-							BackPressureStatsTracker statsTracker = new BackPressureStatsTracker(
-								coordinator,
-								100 * 1000,
-								20,
-								Time.milliseconds(10L));
-
-							int numAttempts = 10;
-
-							int nextSampleId = 0;
-
-							// Verify that all tasks are back pressured. This
-							// can fail if the task takes longer to request
-							// the buffer.
-							for (int attempt = 0; attempt < numAttempts; attempt++) {
-								try {
-									OperatorBackPressureStats stats = triggerStatsSample(statsTracker, vertex);
-
-									assertEquals(nextSampleId + attempt, stats.getSampleId());
-									assertEquals(parallelism, stats.getNumberOfSubTasks());
-									assertEquals(1.0, stats.getMaxBackPressureRatio(), 0.0);
-
-									for (int i = 0; i < parallelism; i++) {
-										assertEquals(1.0, stats.getBackPressureRatio(i), 0.0);
-									}
-
-									nextSampleId = stats.getSampleId() + 1;
-
-									break;
-								} catch (Throwable t) {
-									if (attempt == numAttempts - 1) {
-										throw t;
-									} else {
-										Thread.sleep(500);
-									}
-								}
-							}
-
-							//
-							// 2) Release all buffers and let the tasks grab one
-							//
-							for (Buffer buf : buffers) {
-								buf.recycle();
-							}
-
-							// Wait for all buffers to be available. The tasks
-							// grab them and then immediately release them.
-							while (testBufferPool.getNumberOfAvailableMemorySegments() < 100) {
-								Thread.sleep(100);
-							}
-
-							// Verify that no task is back pressured any more.
-							for (int attempt = 0; attempt < numAttempts; attempt++) {
-								try {
-									OperatorBackPressureStats stats = triggerStatsSample(statsTracker, vertex);
-
-									assertEquals(nextSampleId + attempt, stats.getSampleId());
-									assertEquals(parallelism, stats.getNumberOfSubTasks());
-
-									// Verify that no task is back pressured
-									for (int i = 0; i < parallelism; i++) {
-										assertEquals(0.0, stats.getBackPressureRatio(i), 0.0);
-									}
-
-									break;
-								} catch (Throwable t) {
-									if (attempt == numAttempts - 1) {
-										throw t;
-									} else {
-										Thread.sleep(500);
-									}
-								}
-							}
-
-							// Shut down
-							jm.tell(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobGraph.getJobID()), testActor);
-
-							// Cancel job
-							jm.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
-
-							// Response to removal notification
-							expectMsgEquals(true);
-
-							//
-							// 3) Trigger stats for archived job
-							//
-							statsTracker.invalidateOperatorStatsCache();
-							assertFalse("Unexpected trigger", statsTracker.triggerStackTraceSample(vertex));
-
-						} catch (Exception e) {
-							e.printStackTrace();
-							fail(e.getMessage());
-						}
-					}
-				};
-			} finally {
-				TestingUtils.stopActor(jobManger);
-				TestingUtils.stopActor(taskManager);
-
-				highAvailabilityServices.closeAndCleanupAllData();
-
-				for (Buffer buf : buffers) {
-					buf.recycle();
-				}
-
-				testBufferPool.lazyDestroy();
-			}
-		}};
-	}
-
-	/**
-	 * Triggers a new stats sample.
-	 */
-	private OperatorBackPressureStats triggerStatsSample(
-			BackPressureStatsTracker statsTracker,
-			ExecutionJobVertex vertex) throws InterruptedException {
-
-		statsTracker.invalidateOperatorStatsCache();
-		assertTrue("Failed to trigger", statsTracker.triggerStackTraceSample(vertex));
-
-		// Sleep minimum duration
-		Thread.sleep(20 * 10);
-
-		Option<OperatorBackPressureStats> stats;
-
-		// Get the stats
-		while ((stats = statsTracker.getOperatorBackPressureStats(vertex)).isEmpty()) {
-			Thread.sleep(10);
-		}
-
-		return stats.get();
-	}
-
-	/**
-	 * A back pressured producer sharing a {@link BufferPool} with the
-	 * test driver.
-	 */
-	public static class BackPressuredTask extends AbstractInvokable {
-
-		@Override
-		public void invoke() throws Exception {
-			while (true) {
-				Buffer buffer = testBufferPool.requestBufferBlocking();
-				// Got a buffer, yay!
-				buffer.recycle();
-
-				new CountDownLatch(1).await();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerTest.java
deleted file mode 100644
index e99d1b7..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerTest.java
+++ /dev/null
@@ -1,192 +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.webmonitor;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the BackPressureStatsTracker.
- */
-public class BackPressureStatsTrackerTest extends TestLogger {
-
-	/** Tests simple statistics with fake stack traces. */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testTriggerStackTraceSample() throws Exception {
-		CompletableFuture<StackTraceSample> sampleFuture = new CompletableFuture<>();
-
-		StackTraceSampleCoordinator sampleCoordinator = mock(StackTraceSampleCoordinator.class);
-		when(sampleCoordinator.triggerStackTraceSample(
-				any(ExecutionVertex[].class),
-				anyInt(),
-				any(Time.class),
-				anyInt())).thenReturn(sampleFuture);
-
-		ExecutionGraph graph = mock(ExecutionGraph.class);
-		when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-		// Same Thread execution context
-		when(graph.getFutureExecutor()).thenReturn(new Executor() {
-
-			@Override
-			public void execute(Runnable runnable) {
-				runnable.run();
-			}
-		});
-
-		ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
-		when(jobVertex.getJobId()).thenReturn(new JobID());
-		when(jobVertex.getJobVertexId()).thenReturn(new JobVertexID());
-		when(jobVertex.getGraph()).thenReturn(graph);
-		when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-		taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-		taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-		taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-		taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-		int numSamples = 100;
-		Time delayBetweenSamples = Time.milliseconds(100L);
-
-		BackPressureStatsTracker tracker = new BackPressureStatsTracker(
-				sampleCoordinator, 9999, numSamples, delayBetweenSamples);
-
-		// Trigger
-		assertTrue("Failed to trigger", tracker.triggerStackTraceSample(jobVertex));
-
-		verify(sampleCoordinator).triggerStackTraceSample(
-				eq(taskVertices),
-				eq(numSamples),
-				eq(delayBetweenSamples),
-				eq(BackPressureStatsTracker.MAX_STACK_TRACE_DEPTH));
-
-		// Trigger again for pending request, should not fire
-		assertFalse("Unexpected trigger", tracker.triggerStackTraceSample(jobVertex));
-
-		assertTrue(tracker.getOperatorBackPressureStats(jobVertex).isEmpty());
-
-		verify(sampleCoordinator).triggerStackTraceSample(
-				eq(taskVertices),
-				eq(numSamples),
-				eq(delayBetweenSamples),
-				eq(BackPressureStatsTracker.MAX_STACK_TRACE_DEPTH));
-
-		assertTrue(tracker.getOperatorBackPressureStats(jobVertex).isEmpty());
-
-		// Complete the future
-		Map<ExecutionAttemptID, List<StackTraceElement[]>> traces = new HashMap<>();
-		for (ExecutionVertex vertex : taskVertices) {
-			List<StackTraceElement[]> taskTraces = new ArrayList<>();
-
-			for (int i = 0; i < taskVertices.length; i++) {
-				// Traces until sub task index are back pressured
-				taskTraces.add(createStackTrace(i <= vertex.getParallelSubtaskIndex()));
-			}
-
-			traces.put(vertex.getCurrentExecutionAttempt().getAttemptId(), taskTraces);
-		}
-
-		int sampleId = 1231;
-		int endTime = 841;
-
-		StackTraceSample sample = new StackTraceSample(
-				sampleId,
-				0,
-				endTime,
-				traces);
-
-		// Succeed the promise
-		sampleFuture.complete(sample);
-
-		assertTrue(tracker.getOperatorBackPressureStats(jobVertex).isDefined());
-
-		OperatorBackPressureStats stats = tracker.getOperatorBackPressureStats(jobVertex).get();
-
-		// Verify the stats
-		assertEquals(sampleId, stats.getSampleId());
-		assertEquals(endTime, stats.getEndTimestamp());
-		assertEquals(taskVertices.length, stats.getNumberOfSubTasks());
-
-		for (int i = 0; i < taskVertices.length; i++) {
-			double ratio = stats.getBackPressureRatio(i);
-			// Traces until sub task index are back pressured
-			assertEquals((i + 1) / ((double) 4), ratio, 0.0);
-		}
-	}
-
-	private StackTraceElement[] createStackTrace(boolean isBackPressure) {
-		if (isBackPressure) {
-			return new StackTraceElement[] { new StackTraceElement(
-					BackPressureStatsTracker.EXPECTED_CLASS_NAME,
-					BackPressureStatsTracker.EXPECTED_METHOD_NAME,
-					"LocalBufferPool.java",
-					133) };
-		} else {
-			return Thread.currentThread().getStackTrace();
-		}
-	}
-
-	private ExecutionVertex mockExecutionVertex(
-			ExecutionJobVertex jobVertex,
-			int subTaskIndex) {
-
-		Execution exec = mock(Execution.class);
-		when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID());
-
-		JobVertexID id = jobVertex.getJobVertexId();
-
-		ExecutionVertex vertex = mock(ExecutionVertex.class);
-		when(vertex.getJobvertexId()).thenReturn(id);
-		when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
-		when(vertex.getParallelSubtaskIndex()).thenReturn(subTaskIndex);
-
-		return vertex;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
deleted file mode 100644
index bd12668..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
+++ /dev/null
@@ -1,203 +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.webmonitor;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.client.JobClient;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
-import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
-import static org.junit.Assert.fail;
-
-/**
- * Simple stack trace sampling test.
- */
-public class StackTraceSampleCoordinatorITCase extends TestLogger {
-
-	private static ActorSystem testActorSystem;
-
-	@BeforeClass
-	public static void setup() {
-		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(testActorSystem);
-	}
-
-	/**
-	 * Tests that a cleared task is answered with a partial success response.
-	 */
-	@Test
-	public void testTaskClearedWhileSampling() throws Exception {
-		new JavaTestKit(testActorSystem) {{
-			final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS);
-
-			// The JobGraph
-			final JobGraph jobGraph = new JobGraph();
-			final int parallelism = 1;
-
-			final JobVertex task = new JobVertex("Task");
-			task.setInvokableClass(BlockingNoOpInvokable.class);
-			task.setParallelism(parallelism);
-
-			jobGraph.addVertex(task);
-
-			final Configuration config = new Configuration();
-
-			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
-				config,
-				TestingUtils.defaultExecutor());
-
-			ActorGateway jobManger = null;
-			ActorGateway taskManager = null;
-
-			try {
-				jobManger = TestingUtils.createJobManager(
-					testActorSystem,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					highAvailabilityServices);
-
-				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
-
-				taskManager = TestingUtils.createTaskManager(
-					testActorSystem,
-					highAvailabilityServices,
-					config,
-					true,
-					true);
-
-				final ActorGateway jm = jobManger;
-
-				new Within(deadline) {
-					@Override
-					protected void run() {
-						try {
-							ActorGateway testActor = new AkkaActorGateway(getTestActor(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-
-							int maxAttempts = 10;
-							int sleepTime = 100;
-
-							for (int i = 0; i < maxAttempts; i++, sleepTime *= 2) {
-								// Submit the job and wait until it is running
-								JobClient.submitJobDetached(
-										new AkkaJobManagerGateway(jm),
-										config,
-										jobGraph,
-										Time.milliseconds(deadline.toMillis()),
-										ClassLoader.getSystemClassLoader());
-
-								jm.tell(new WaitForAllVerticesToBeRunning(jobGraph.getJobID()), testActor);
-
-								expectMsgEquals(new AllVerticesRunning(jobGraph.getJobID()));
-
-								// Get the ExecutionGraph
-								jm.tell(new RequestExecutionGraph(jobGraph.getJobID()), testActor);
-								ExecutionGraphFound executionGraphResponse =
-										expectMsgClass(ExecutionGraphFound.class);
-								ExecutionGraph executionGraph = (ExecutionGraph) executionGraphResponse.executionGraph();
-								ExecutionJobVertex vertex = executionGraph.getJobVertex(task.getID());
-
-								StackTraceSampleCoordinator coordinator = new StackTraceSampleCoordinator(
-										testActorSystem.dispatcher(), 60000);
-
-								CompletableFuture<StackTraceSample> sampleFuture = coordinator.triggerStackTraceSample(
-									vertex.getTaskVertices(),
-									// Do this often so we have a good
-									// chance of removing the job during
-									// sampling.
-									21474700 * 100,
-									Time.milliseconds(10L),
-									0);
-
-								// Wait before cancelling so that some samples
-								// are actually taken.
-								Thread.sleep(sleepTime);
-
-								// Cancel job
-								Future<?> removeFuture = jm.ask(
-										new TestingJobManagerMessages.NotifyWhenJobRemoved(jobGraph.getJobID()),
-										remaining());
-
-								jm.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
-
-								try {
-									// Throws Exception on failure
-									sampleFuture.get(remaining().toMillis(), TimeUnit.MILLISECONDS);
-
-									// OK, we are done. Got the expected
-									// partial result.
-									break;
-								} catch (Throwable t) {
-									// We were too fast in cancelling the job.
-									// Fall through and retry.
-								} finally {
-									Await.ready(removeFuture, remaining());
-								}
-							}
-						} catch (Exception e) {
-							e.printStackTrace();
-							fail(e.getMessage());
-						}
-					}
-				};
-			} finally {
-				TestingUtils.stopActor(jobManger);
-				TestingUtils.stopActor(taskManager);
-
-				highAvailabilityServices.closeAndCleanupAllData();
-			}
-		}};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorTest.java
deleted file mode 100644
index 08c4212..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorTest.java
+++ /dev/null
@@ -1,441 +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.webmonitor;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample;
-import org.apache.flink.runtime.messages.StackTraceSampleResponse;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorSystem;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-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;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Test for the {@link StackTraceSampleCoordinator}.
- */
-public class StackTraceSampleCoordinatorTest extends TestLogger {
-
-	private static ActorSystem system;
-
-	private StackTraceSampleCoordinator coord;
-
-	@BeforeClass
-	public static void setUp() throws Exception {
-		system = AkkaUtils.createLocalActorSystem(new Configuration());
-	}
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (system != null) {
-			system.shutdown();
-		}
-	}
-
-	@Before
-	public void init() throws Exception {
-		this.coord = new StackTraceSampleCoordinator(system.dispatcher(), 60000);
-	}
-
-	/** Tests simple trigger and collect of stack trace samples. */
-	@Test
-	public void testTriggerStackTraceSample() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true)
-		};
-
-		int numSamples = 1;
-		Time delayBetweenSamples = Time.milliseconds(100L);
-		int maxStackTraceDepth = 0;
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-				vertices, numSamples, delayBetweenSamples, maxStackTraceDepth);
-
-		// Verify messages have been sent
-		for (ExecutionVertex vertex : vertices) {
-			ExecutionAttemptID expectedExecutionId = vertex
-					.getCurrentExecutionAttempt().getAttemptId();
-
-			TriggerStackTraceSample expectedMsg = new TriggerStackTraceSample(
-					0,
-					expectedExecutionId,
-					numSamples,
-					delayBetweenSamples,
-					maxStackTraceDepth);
-
-			verify(vertex.getCurrentExecutionAttempt())
-				.requestStackTraceSample(eq(0), eq(numSamples), eq(delayBetweenSamples), eq(maxStackTraceDepth), any(Time.class));
-		}
-
-		assertFalse(sampleFuture.isDone());
-
-		StackTraceElement[] stackTraceSample = Thread.currentThread().getStackTrace();
-		List<StackTraceElement[]> traces = new ArrayList<>();
-		traces.add(stackTraceSample);
-		traces.add(stackTraceSample);
-		traces.add(stackTraceSample);
-
-		// Collect stack traces
-		for (int i = 0; i < vertices.length; i++) {
-			ExecutionAttemptID executionId = vertices[i].getCurrentExecutionAttempt().getAttemptId();
-			coord.collectStackTraces(0, executionId, traces);
-
-			if (i == vertices.length - 1) {
-				assertTrue(sampleFuture.isDone());
-			} else {
-				assertFalse(sampleFuture.isDone());
-			}
-		}
-
-		// Verify completed stack trace sample
-		StackTraceSample sample = sampleFuture.get();
-
-		assertEquals(0, sample.getSampleId());
-		assertTrue(sample.getEndTime() >= sample.getStartTime());
-
-		Map<ExecutionAttemptID, List<StackTraceElement[]>> tracesByTask = sample.getStackTraces();
-
-		for (ExecutionVertex vertex : vertices) {
-			ExecutionAttemptID executionId = vertex.getCurrentExecutionAttempt().getAttemptId();
-			List<StackTraceElement[]> sampleTraces = tracesByTask.get(executionId);
-
-			assertNotNull("Task not found", sampleTraces);
-			assertTrue(traces.equals(sampleTraces));
-		}
-
-		// Verify no more pending sample
-		assertEquals(0, coord.getNumberOfPendingSamples());
-
-		// Verify no error on late collect
-		coord.collectStackTraces(0, vertices[0].getCurrentExecutionAttempt().getAttemptId(), traces);
-	}
-
-	/** Tests triggering for non-running tasks fails the future. */
-	@Test
-	public void testTriggerStackTraceSampleNotRunningTasks() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.DEPLOYING, true)
-		};
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-			vertices,
-			1,
-			Time.milliseconds(100L),
-			0);
-
-		assertTrue(sampleFuture.isDone());
-
-		try {
-			sampleFuture.get();
-			fail("Expected exception.");
-		} catch (ExecutionException e) {
-			assertTrue(e.getCause() instanceof IllegalStateException);
-		}
-	}
-
-	/** Tests triggering for reset tasks fails the future. */
-	@Test(timeout = 1000L)
-	public void testTriggerStackTraceSampleResetRunningTasks() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-				// Fails to send the message to the execution (happens when execution is reset)
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, false)
-		};
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-			vertices,
-			1,
-			Time.milliseconds(100L),
-			0);
-
-		try {
-			sampleFuture.get();
-			fail("Expected exception.");
-		} catch (ExecutionException e) {
-			assertTrue(e.getCause() instanceof RuntimeException);
-		}
-	}
-
-	/** Tests that samples time out if they don't finish in time. */
-	@Test(timeout = 1000L)
-	public void testTriggerStackTraceSampleTimeout() throws Exception {
-		int timeout = 100;
-
-		coord = new StackTraceSampleCoordinator(system.dispatcher(), timeout);
-
-		final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
-
-		try {
-
-			ExecutionVertex[] vertices = new ExecutionVertex[]{
-				mockExecutionVertexWithTimeout(
-					new ExecutionAttemptID(),
-					ExecutionState.RUNNING,
-					scheduledExecutorService,
-					timeout)
-			};
-
-			CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0);
-
-			// Wait for the timeout
-			Thread.sleep(timeout * 2);
-
-			boolean success = false;
-			for (int i = 0; i < 10; i++) {
-				if (sampleFuture.isDone()) {
-					success = true;
-					break;
-				}
-
-				Thread.sleep(timeout);
-			}
-
-			assertTrue("Sample did not time out", success);
-
-			try {
-				sampleFuture.get();
-				fail("Expected exception.");
-			} catch (ExecutionException e) {
-				assertTrue(e.getCause().getCause().getMessage().contains("Timeout"));
-			}
-
-			// Collect after the timeout (should be ignored)
-			ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
-			coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
-		} finally {
-			scheduledExecutorService.shutdownNow();
-		}
-	}
-
-	/** Tests that collecting an unknown sample is ignored. */
-	@Test
-	public void testCollectStackTraceForUnknownSample() throws Exception {
-		coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList<StackTraceElement[]>());
-	}
-
-	/** Tests cancelling of a pending sample. */
-	@Test
-	public void testCancelStackTraceSample() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-		};
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0);
-
-		assertFalse(sampleFuture.isDone());
-
-		// Cancel
-		coord.cancelStackTraceSample(0, null);
-
-		// Verify completed
-		assertTrue(sampleFuture.isDone());
-
-		// Verify no more pending samples
-		assertEquals(0, coord.getNumberOfPendingSamples());
-	}
-
-	/** Tests that collecting for a cancelled sample throws no Exception. */
-	@Test
-	public void testCollectStackTraceForCanceledSample() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-		};
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0);
-
-		assertFalse(sampleFuture.isDone());
-
-		coord.cancelStackTraceSample(0, null);
-
-		assertTrue(sampleFuture.isDone());
-
-		// Verify no error on late collect
-		ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
-		coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
-	}
-
-	/** Tests that collecting for a cancelled sample throws no Exception. */
-	@Test
-	public void testCollectForDiscardedPendingSample() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-		};
-
-		CompletableFuture<StackTraceSample> sampleFuture = coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0);
-
-		assertFalse(sampleFuture.isDone());
-
-		coord.cancelStackTraceSample(0, null);
-
-		assertTrue(sampleFuture.isDone());
-
-		// Verify no error on late collect
-		ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
-		coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
-	}
-
-
-	/** Tests that collecting for a unknown task fails. */
-	@Test(expected = IllegalArgumentException.class)
-	public void testCollectStackTraceForUnknownTask() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-		};
-
-		coord.triggerStackTraceSample(vertices, 1, Time.milliseconds(100L), 0);
-
-		coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList<StackTraceElement[]>());
-	}
-
-	/** Tests that shut down fails all pending samples and future sample triggers. */
-	@Test
-	public void testShutDown() throws Exception {
-		ExecutionVertex[] vertices = new ExecutionVertex[] {
-				mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true),
-		};
-
-		List<CompletableFuture<StackTraceSample>> sampleFutures = new ArrayList<>();
-
-		// Trigger
-		sampleFutures.add(coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0));
-
-		sampleFutures.add(coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0));
-
-		for (CompletableFuture<StackTraceSample> future : sampleFutures) {
-			assertFalse(future.isDone());
-		}
-
-		// Shut down
-		coord.shutDown();
-
-		// Verify all completed
-		for (CompletableFuture<StackTraceSample> future : sampleFutures) {
-			assertTrue(future.isDone());
-		}
-
-		// Verify new trigger returns failed future
-		CompletableFuture<StackTraceSample> future = coord.triggerStackTraceSample(
-				vertices, 1, Time.milliseconds(100L), 0);
-
-		assertTrue(future.isDone());
-
-		try {
-			future.get();
-			fail("Expected exception.");
-		} catch (ExecutionException e) {
-			// we expected an exception here :-)
-		}
-
-	}
-
-	// ------------------------------------------------------------------------
-
-	private ExecutionVertex mockExecutionVertex(
-			ExecutionAttemptID executionId,
-			ExecutionState state,
-			boolean sendSuccess) {
-
-		Execution exec = mock(Execution.class);
-		CompletableFuture<StackTraceSampleResponse> failedFuture = new CompletableFuture<>();
-		failedFuture.completeExceptionally(new Exception("Send failed."));
-
-		when(exec.getAttemptId()).thenReturn(executionId);
-		when(exec.getState()).thenReturn(state);
-		when(exec.requestStackTraceSample(anyInt(), anyInt(), any(Time.class), anyInt(), any(Time.class)))
-			.thenReturn(
-				sendSuccess ?
-					CompletableFuture.completedFuture(mock(StackTraceSampleResponse.class)) :
-					failedFuture);
-
-		ExecutionVertex vertex = mock(ExecutionVertex.class);
-		when(vertex.getJobvertexId()).thenReturn(new JobVertexID());
-		when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
-
-		return vertex;
-	}
-
-	private ExecutionVertex mockExecutionVertexWithTimeout(
-		ExecutionAttemptID executionId,
-		ExecutionState state,
-		ScheduledExecutorService scheduledExecutorService,
-		int timeout) {
-
-		final CompletableFuture<StackTraceSampleResponse> future = new CompletableFuture<>();
-
-		Execution exec = mock(Execution.class);
-		when(exec.getAttemptId()).thenReturn(executionId);
-		when(exec.getState()).thenReturn(state);
-		when(exec.requestStackTraceSample(anyInt(), anyInt(), any(Time.class), anyInt(), any(Time.class)))
-			.thenReturn(future);
-
-		scheduledExecutorService.schedule(new Runnable() {
-			@Override
-			public void run() {
-				future.completeExceptionally(new TimeoutException("Timeout"));
-			}
-		}, timeout, TimeUnit.MILLISECONDS);
-
-		ExecutionVertex vertex = mock(ExecutionVertex.class);
-		when(vertex.getJobvertexId()).thenReturn(new JobVertexID());
-		when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
-
-		return vertex;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/files/MimeTypesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/files/MimeTypesTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/files/MimeTypesTest.java
deleted file mode 100644
index 0a8d9d8..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/files/MimeTypesTest.java
+++ /dev/null
@@ -1,75 +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.webmonitor.files;
-
-import org.apache.flink.runtime.rest.handler.util.MimeTypes;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for the MIME types map.
- */
-public class MimeTypesTest {
-
-	@Test
-	public void testCompleteness() {
-		try {
-			assertNotNull(MimeTypes.getMimeTypeForExtension("txt"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("htm"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("html"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("css"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("js"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("json"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("png"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("jpg"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("jpeg"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("gif"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("woff"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("woff2"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("otf"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("ttf"));
-			assertNotNull(MimeTypes.getMimeTypeForExtension("eot"));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testFileNameExtraction() {
-		try {
-			assertNotNull(MimeTypes.getMimeTypeForFileName("test.txt"));
-			assertNotNull(MimeTypes.getMimeTypeForFileName("t.txt"));
-			assertNotNull(MimeTypes.getMimeTypeForFileName("first.second.third.txt"));
-
-			assertNull(MimeTypes.getMimeTypeForFileName(".txt"));
-			assertNull(MimeTypes.getMimeTypeForFileName("txt"));
-			assertNull(MimeTypes.getMimeTypeForFileName("test."));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java
deleted file mode 100644
index 69ee762..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java
+++ /dev/null
@@ -1,38 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.Executors;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the ClusterOverviewHandler.
- */
-public class ClusterOverviewHandlerTest {
-	@Test
-	public void testGetPaths() {
-		ClusterOverviewHandler handler = new ClusterOverviewHandler(Executors.directExecutor(), Time.seconds(0L));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/overview", paths[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java
deleted file mode 100644
index 6061e4b..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java
+++ /dev/null
@@ -1,38 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.Executors;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the CurrentJobIdsHandler.
- */
-public class CurrentJobIdsHandlerTest {
-	@Test
-	public void testGetPaths() {
-		CurrentJobIdsHandler handler = new CurrentJobIdsHandler(Executors.directExecutor(), Time.seconds(0L));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs", paths[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java
deleted file mode 100644
index ccfafd4..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java
+++ /dev/null
@@ -1,121 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.messages.webmonitor.JobDetails;
-import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Collection;
-
-/**
- * Tests for the CurrentJobsOverviewHandler.
- */
-public class CurrentJobsOverviewHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new CurrentJobsOverviewHandler.CurrentJobsOverviewJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		JobDetails expectedDetails = WebMonitorUtils.createDetailsForJob(originalJob);
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/joboverview", archive.getPath());
-
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(archive.getJson());
-		ArrayNode running = (ArrayNode) result.get("running");
-		Assert.assertEquals(0, running.size());
-
-		ArrayNode finished = (ArrayNode) result.get("finished");
-		Assert.assertEquals(1, finished.size());
-
-		compareJobOverview(expectedDetails, finished.get(0).toString());
-	}
-
-	@Test
-	public void testGetPaths() {
-		CurrentJobsOverviewHandler handlerAll = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, true);
-		String[] pathsAll = handlerAll.getPaths();
-		Assert.assertEquals(1, pathsAll.length);
-		Assert.assertEquals("/joboverview", pathsAll[0]);
-
-		CurrentJobsOverviewHandler handlerRunning = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, false);
-		String[] pathsRunning = handlerRunning.getPaths();
-		Assert.assertEquals(1, pathsRunning.length);
-		Assert.assertEquals("/joboverview/running", pathsRunning[0]);
-
-		CurrentJobsOverviewHandler handlerCompleted = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), false, true);
-		String[] pathsCompleted = handlerCompleted.getPaths();
-		Assert.assertEquals(1, pathsCompleted.length);
-		Assert.assertEquals("/joboverview/completed", pathsCompleted[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		JobDetails expectedDetails = WebMonitorUtils.createDetailsForJob(originalJob);
-		StringWriter writer = new StringWriter();
-		try (JsonGenerator gen = ArchivedJobGenerationUtils.JACKSON_FACTORY.createGenerator(writer)) {
-			CurrentJobsOverviewHandler.writeJobDetailOverviewAsJson(expectedDetails, gen, 0);
-		}
-		compareJobOverview(expectedDetails, writer.toString());
-	}
-
-	private static void compareJobOverview(JobDetails expectedDetails, String answer) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
-
-		Assert.assertEquals(expectedDetails.getJobId().toString(), result.get("jid").asText());
-		Assert.assertEquals(expectedDetails.getJobName(), result.get("name").asText());
-		Assert.assertEquals(expectedDetails.getStatus().name(), result.get("state").asText());
-
-		Assert.assertEquals(expectedDetails.getStartTime(), result.get("start-time").asLong());
-		Assert.assertEquals(expectedDetails.getEndTime(), result.get("end-time").asLong());
-		Assert.assertEquals(expectedDetails.getEndTime() - expectedDetails.getStartTime(), result.get("duration").asLong());
-		Assert.assertEquals(expectedDetails.getLastUpdateTime(), result.get("last-modification").asLong());
-
-		JsonNode tasks = result.get("tasks");
-		Assert.assertEquals(expectedDetails.getNumTasks(), tasks.get("total").asInt());
-		int[] tasksPerState = expectedDetails.getNumVerticesPerExecutionState();
-		Assert.assertEquals(
-			tasksPerState[ExecutionState.CREATED.ordinal()] + tasksPerState[ExecutionState.SCHEDULED.ordinal()] + tasksPerState[ExecutionState.DEPLOYING.ordinal()],
-			tasks.get("pending").asInt());
-		Assert.assertEquals(tasksPerState[ExecutionState.RUNNING.ordinal()], tasks.get("running").asInt());
-		Assert.assertEquals(tasksPerState[ExecutionState.FINISHED.ordinal()], tasks.get("finished").asInt());
-		Assert.assertEquals(tasksPerState[ExecutionState.CANCELING.ordinal()], tasks.get("canceling").asInt());
-		Assert.assertEquals(tasksPerState[ExecutionState.CANCELED.ordinal()], tasks.get("canceled").asInt());
-		Assert.assertEquals(tasksPerState[ExecutionState.FAILED.ordinal()], tasks.get("failed").asInt());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java
deleted file mode 100644
index 22b3e5e..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java
+++ /dev/null
@@ -1,59 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.TimeZone;
-
-/**
- * Tests for the DashboardConfigHandler.
- */
-public class DashboardConfigHandlerTest {
-	@Test
-	public void testGetPaths() {
-		DashboardConfigHandler handler = new DashboardConfigHandler(Executors.directExecutor(), 10000L);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/config", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		long refreshInterval = 12345;
-		TimeZone timeZone = TimeZone.getDefault();
-		EnvironmentInformation.RevisionInformation revision = EnvironmentInformation.getRevisionInformation();
-
-		String json = DashboardConfigHandler.createConfigJson(refreshInterval);
-
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(refreshInterval, result.get("refresh-interval").asLong());
-		Assert.assertEquals(timeZone.getDisplayName(), result.get("timezone-name").asText());
-		Assert.assertEquals(timeZone.getRawOffset(), result.get("timezone-offset").asLong());
-		Assert.assertEquals(EnvironmentInformation.getVersion(), result.get("flink-version").asText());
-		Assert.assertEquals(revision.commitId + " @ " + revision.commitDate, result.get("flink-revision").asText());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
deleted file mode 100644
index e79be96..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
+++ /dev/null
@@ -1,74 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.akka.AkkaJobManagerGateway;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.rest.handler.util.HandlerRedirectUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the HandlerRedirectUtils.
- */
-public class HandlerRedirectUtilsTest extends TestLogger {
-
-	private static final String localRestAddress = "http://127.0.0.1:1234";
-	private static final String remoteRestAddress = "http://127.0.0.2:1234";
-
-	@Test
-	public void testGetRedirectAddressWithLocalEqualsRemoteRESTAddress() throws Exception {
-		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
-		when(jobManagerGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(localRestAddress));
-
-		CompletableFuture<Optional<String>> redirectingAddressFuture = HandlerRedirectUtils.getRedirectAddress(
-			localRestAddress,
-			jobManagerGateway,
-			Time.seconds(3L));
-
-		Assert.assertTrue(redirectingAddressFuture.isDone());
-		// no redirection needed
-		Assert.assertFalse(redirectingAddressFuture.get().isPresent());
-	}
-
-	@Test
-	public void testGetRedirectAddressWithRemoteAkkaPath() throws Exception {
-		JobManagerGateway jobManagerGateway = mock(AkkaJobManagerGateway.class);
-		when(jobManagerGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(remoteRestAddress));
-
-		CompletableFuture<Optional<String>> optRedirectingAddress = HandlerRedirectUtils.getRedirectAddress(
-			localRestAddress,
-			jobManagerGateway,
-			Time.seconds(3L));
-
-		Assert.assertTrue(optRedirectingAddress.isDone());
-
-		Assert.assertEquals(remoteRestAddress, optRedirectingAddress.get().get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandlerTest.java
index 647e782..eb0f6b3 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandlerTest.java
@@ -19,15 +19,14 @@
 package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
-import org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.JarActionHandlerConfig;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.HashMap;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 
 /**
  * Tests for the JarActionHandler.
@@ -49,7 +48,7 @@ public class JarActionHandlerTest {
 		Map<String, String> queryParams = new HashMap<>(); // <-- everything goes here
 
 		// Nothing configured
-		JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
+		JarActionHandler.JarActionHandlerConfig config = JarActionHandler.JarActionHandlerConfig.fromParams(pathParams, queryParams);
 		assertEquals(SavepointRestoreSettings.none(), config.getSavepointRestoreSettings());
 
 		// Set path
@@ -58,14 +57,14 @@ public class JarActionHandlerTest {
 
 		SavepointRestoreSettings expected = SavepointRestoreSettings.forPath("the-savepoint-path", false);
 
-		config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
+		config = JarActionHandler.JarActionHandlerConfig.fromParams(pathParams, queryParams);
 		assertEquals(expected, config.getSavepointRestoreSettings());
 
 		// Set flag
 		queryParams.put("allowNonRestoredState", "true");
 
 		expected = SavepointRestoreSettings.forPath("the-savepoint-path", true);
-		config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
+		config = JarActionHandler.JarActionHandlerConfig.fromParams(pathParams, queryParams);
 		assertEquals(expected, config.getSavepointRestoreSettings());
 	}
 
@@ -85,10 +84,10 @@ public class JarActionHandlerTest {
 		queryParams.put("allowNonRestoredState", "");
 
 		// Nothing configured
-		JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
+		JarActionHandler.JarActionHandlerConfig config = JarActionHandler.JarActionHandlerConfig.fromParams(pathParams, queryParams);
 
 		assertEquals(0, config.getProgramArgs().length);
-		assertNull(config.getEntryClass());
+		Assert.assertNull(config.getEntryClass());
 		assertEquals(1, config.getParallelism());
 		assertEquals(SavepointRestoreSettings.none(), config.getSavepointRestoreSettings());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java
deleted file mode 100644
index 5510fed..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java
+++ /dev/null
@@ -1,83 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobAccumulatorsHandler.
- */
-public class JobAccumulatorsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobAccumulatorsHandler.JobAccumulatorsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/accumulators", archive.getPath());
-		compareAccumulators(originalJob, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobAccumulatorsHandler handler = new JobAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/accumulators", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		String json = JobAccumulatorsHandler.createJobAccumulatorsJson(originalJob);
-
-		compareAccumulators(originalJob, json);
-	}
-
-	private static void compareAccumulators(AccessExecutionGraph originalJob, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		ArrayNode accs = (ArrayNode) result.get("job-accumulators");
-		Assert.assertEquals(0, accs.size());
-
-		Assert.assertTrue(originalJob.getAccumulatorResultsStringified().length > 0);
-		ArchivedJobGenerationUtils.compareStringifiedAccumulators(
-			originalJob.getAccumulatorResultsStringified(),
-			(ArrayNode) result.get("user-task-accumulators"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java
deleted file mode 100644
index 86c5295..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java
+++ /dev/null
@@ -1,44 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-
-import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.List;
-
-/**
- * Tests for the JobCancellationHandler.
- */
-public class JobCancellationHandlerTest {
-	@Test
-	public void testGetPaths() {
-		JobCancellationHandler handler = new JobCancellationHandler(Executors.directExecutor(), TestingUtils.TIMEOUT());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(2, paths.length);
-		List<String> pathsList = Lists.newArrayList(paths);
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid/cancel"));
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid/yarn-cancel"));
-	}
-}


[10/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java
deleted file mode 100644
index 31c2212..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java
+++ /dev/null
@@ -1,103 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the SubtasksTimesHandler.
- */
-public class SubtasksTimesHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new SubtasksTimesHandler.SubtasksTimesJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/subtasktimes", archive.getPath());
-		compareSubtaskTimes(originalTask, originalAttempt, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		SubtasksTimesHandler handler = new SubtasksTimesHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasktimes", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-		String json = SubtasksTimesHandler.createSubtaskTimesJson(originalTask);
-
-		compareSubtaskTimes(originalTask, originalAttempt, json);
-	}
-
-	private static void compareSubtaskTimes(AccessExecutionJobVertex originalTask, AccessExecution originalAttempt, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
-		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
-		Assert.assertTrue(result.get("now").asLong() > 0L);
-
-		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
-
-		JsonNode subtask = subtasks.get(0);
-		Assert.assertEquals(0, subtask.get("subtask").asInt());
-		Assert.assertEquals(originalAttempt.getAssignedResourceLocation().getHostname(), subtask.get("host").asText());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(originalAttempt.getState()) - originalAttempt.getStateTimestamp(ExecutionState.SCHEDULED), subtask.get("duration").asLong());
-
-		JsonNode timestamps = subtask.get("timestamps");
-
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CREATED), timestamps.get(ExecutionState.CREATED.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.SCHEDULED), timestamps.get(ExecutionState.SCHEDULED.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.DEPLOYING), timestamps.get(ExecutionState.DEPLOYING.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.RUNNING), timestamps.get(ExecutionState.RUNNING.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.FINISHED), timestamps.get(ExecutionState.FINISHED.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CANCELING), timestamps.get(ExecutionState.CANCELING.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.CANCELED), timestamps.get(ExecutionState.CANCELED.name()).asLong());
-		Assert.assertEquals(originalAttempt.getStateTimestamp(ExecutionState.FAILED), timestamps.get(ExecutionState.FAILED.name()).asLong());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
deleted file mode 100644
index faeff13..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
+++ /dev/null
@@ -1,149 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.blob.BlobKey;
-import org.apache.flink.runtime.blob.VoidBlobStore;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.isA;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-/**
- * Tests for the TaskManagersLogHandler.
- */
-public class TaskManagerLogHandlerTest {
-	@Test
-	public void testGetPaths() {
-		TaskManagerLogHandler handlerLog = new TaskManagerLogHandler(
-			mock(GatewayRetriever.class),
-			Executors.directExecutor(),
-			CompletableFuture.completedFuture("/jm/address"),
-			TestingUtils.TIMEOUT(),
-			TaskManagerLogHandler.FileMode.LOG,
-			new Configuration(),
-			new VoidBlobStore());
-		String[] pathsLog = handlerLog.getPaths();
-		Assert.assertEquals(1, pathsLog.length);
-		Assert.assertEquals("/taskmanagers/:taskmanagerid/log", pathsLog[0]);
-
-		TaskManagerLogHandler handlerOut = new TaskManagerLogHandler(
-			mock(GatewayRetriever.class),
-			Executors.directExecutor(),
-			CompletableFuture.completedFuture("/jm/address"),
-			TestingUtils.TIMEOUT(),
-			TaskManagerLogHandler.FileMode.STDOUT,
-			new Configuration(),
-			new VoidBlobStore());
-		String[] pathsOut = handlerOut.getPaths();
-		Assert.assertEquals(1, pathsOut.length);
-		Assert.assertEquals("/taskmanagers/:taskmanagerid/stdout", pathsOut[0]);
-	}
-
-	@Test
-	public void testLogFetchingFailure() throws Exception {
-		// ========= setup TaskManager =================================================================================
-		InstanceID tmID = new InstanceID();
-		ResourceID tmRID = new ResourceID(tmID.toString());
-		TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class);
-		when(taskManagerGateway.getAddress()).thenReturn("/tm/address");
-
-		Instance taskManager = mock(Instance.class);
-		when(taskManager.getId()).thenReturn(tmID);
-		when(taskManager.getTaskManagerID()).thenReturn(tmRID);
-		when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway);
-		CompletableFuture<BlobKey> future = new CompletableFuture<>();
-		future.completeExceptionally(new IOException("failure"));
-		when(taskManagerGateway.requestTaskManagerLog(any(Time.class))).thenReturn(future);
-
-		// ========= setup JobManager ==================================================================================
-
-		JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class);
-		when(jobManagerGateway.requestBlobServerPort(any(Time.class))).thenReturn(CompletableFuture.completedFuture(1337));
-		when(jobManagerGateway.getHostname()).thenReturn("localhost");
-		when(jobManagerGateway.requestTaskManagerInstance(any(InstanceID.class), any(Time.class))).thenReturn(
-			CompletableFuture.completedFuture(Optional.of(taskManager)));
-
-		GatewayRetriever<JobManagerGateway> retriever = mock(GatewayRetriever.class);
-		when(retriever.getNow())
-			.thenReturn(Optional.of(jobManagerGateway));
-
-		TaskManagerLogHandler handler = new TaskManagerLogHandler(
-			retriever,
-			Executors.directExecutor(),
-			CompletableFuture.completedFuture("/jm/address"),
-			TestingUtils.TIMEOUT(),
-			TaskManagerLogHandler.FileMode.LOG,
-			new Configuration(),
-			new VoidBlobStore());
-
-		final AtomicReference<String> exception = new AtomicReference<>();
-
-		ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-		when(ctx.write(isA(ByteBuf.class))).thenAnswer(new Answer<Object>() {
-			@Override
-			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
-				ByteBuf data = invocationOnMock.getArgumentAt(0, ByteBuf.class);
-				exception.set(new String(data.array(), ConfigConstants.DEFAULT_CHARSET));
-				return null;
-			}
-		});
-
-		Map<String, String> pathParams = new HashMap<>();
-		pathParams.put(TaskManagersHandler.TASK_MANAGER_ID_KEY, tmID.toString());
-		Routed routed = mock(Routed.class);
-		when(routed.pathParams()).thenReturn(pathParams);
-		when(routed.request()).thenReturn(new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/taskmanagers/" + tmID + "/log"));
-
-		handler.respondAsLeader(ctx, routed, jobManagerGateway);
-
-		Assert.assertEquals("Fetching TaskManager log failed.", exception.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java
deleted file mode 100644
index e3a71a1..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java
+++ /dev/null
@@ -1,44 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.concurrent.Executors;
-
-import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.List;
-
-/**
- * Tests for the TaskManagersHandler.
- */
-public class TaskManagersHandlerTest {
-	@Test
-	public void testGetPaths() {
-		TaskManagersHandler handler = new TaskManagersHandler(Executors.directExecutor(), Time.seconds(0L), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(2, paths.length);
-		List<String> pathsList = Lists.newArrayList(paths);
-		Assert.assertTrue(pathsList.contains("/taskmanagers"));
-		Assert.assertTrue(pathsList.contains("/taskmanagers/:taskmanagerid"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
deleted file mode 100644
index 47298be..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java
+++ /dev/null
@@ -1,195 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
-import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the CheckpointConfigHandler.
- */
-public class CheckpointConfigHandlerTest {
-
-	@Test
-	public void testArchiver() throws IOException {
-		JsonArchivist archivist = new CheckpointConfigHandler.CheckpointConfigJsonArchivist();
-		GraphAndSettings graphAndSettings = createGraphAndSettings(true, true);
-
-		AccessExecutionGraph graph = graphAndSettings.graph;
-		when(graph.getJobID()).thenReturn(new JobID());
-		JobCheckpointingSettings settings = graphAndSettings.snapshottingSettings;
-		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
-		Assert.assertEquals(1, archives.size());
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + graph.getJobID() + "/checkpoints/config", archive.getPath());
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(archive.getJson());
-
-		Assert.assertEquals("exactly_once", rootNode.get("mode").asText());
-		Assert.assertEquals(settings.getCheckpointInterval(), rootNode.get("interval").asLong());
-		Assert.assertEquals(settings.getCheckpointTimeout(), rootNode.get("timeout").asLong());
-		Assert.assertEquals(settings.getMinPauseBetweenCheckpoints(), rootNode.get("min_pause").asLong());
-		Assert.assertEquals(settings.getMaxConcurrentCheckpoints(), rootNode.get("max_concurrent").asInt());
-
-		JsonNode externalizedNode = rootNode.get("externalization");
-		Assert.assertNotNull(externalizedNode);
-		Assert.assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
-		Assert.assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
-
-	}
-
-	@Test
-	public void testGetPaths() {
-		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/checkpoints/config", paths[0]);
-	}
-
-	/**
-	 * Tests a simple config.
-	 */
-	@Test
-	public void testSimpleConfig() throws Exception {
-		GraphAndSettings graphAndSettings = createGraphAndSettings(false, true);
-
-		AccessExecutionGraph graph = graphAndSettings.graph;
-		JobCheckpointingSettings settings = graphAndSettings.snapshottingSettings;
-
-		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(json);
-
-		assertEquals("exactly_once", rootNode.get("mode").asText());
-		assertEquals(settings.getCheckpointInterval(), rootNode.get("interval").asLong());
-		assertEquals(settings.getCheckpointTimeout(), rootNode.get("timeout").asLong());
-		assertEquals(settings.getMinPauseBetweenCheckpoints(), rootNode.get("min_pause").asLong());
-		assertEquals(settings.getMaxConcurrentCheckpoints(), rootNode.get("max_concurrent").asInt());
-
-		JsonNode externalizedNode = rootNode.get("externalization");
-		assertNotNull(externalizedNode);
-		assertEquals(false, externalizedNode.get("enabled").asBoolean());
-	}
-
-	/**
-	 * Tests the that the isExactlyOnce flag is respected.
-	 */
-	@Test
-	public void testAtLeastOnce() throws Exception {
-		GraphAndSettings graphAndSettings = createGraphAndSettings(false, false);
-
-		AccessExecutionGraph graph = graphAndSettings.graph;
-
-		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(json);
-
-		assertEquals("at_least_once", rootNode.get("mode").asText());
-	}
-
-	/**
-	 * Tests that the externalized checkpoint settings are forwarded.
-	 */
-	@Test
-	public void testEnabledExternalizedCheckpointSettings() throws Exception {
-		GraphAndSettings graphAndSettings = createGraphAndSettings(true, false);
-
-		AccessExecutionGraph graph = graphAndSettings.graph;
-		ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings;
-
-		CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode externalizedNode = mapper.readTree(json).get("externalization");
-		assertNotNull(externalizedNode);
-		assertEquals(externalizedSettings.externalizeCheckpoints(), externalizedNode.get("enabled").asBoolean());
-		assertEquals(externalizedSettings.deleteOnCancellation(), externalizedNode.get("delete_on_cancellation").asBoolean());
-	}
-
-	private static GraphAndSettings createGraphAndSettings(boolean externalized, boolean exactlyOnce) {
-		long interval = 18231823L;
-		long timeout = 996979L;
-		long minPause = 119191919L;
-		int maxConcurrent = 12929329;
-		ExternalizedCheckpointSettings externalizedSetting = externalized
-			? ExternalizedCheckpointSettings.externalizeCheckpoints(true)
-			: ExternalizedCheckpointSettings.none();
-
-		JobCheckpointingSettings settings = new JobCheckpointingSettings(
-			Collections.<JobVertexID>emptyList(),
-			Collections.<JobVertexID>emptyList(),
-			Collections.<JobVertexID>emptyList(),
-			interval,
-			timeout,
-			minPause,
-			maxConcurrent,
-			externalizedSetting,
-			null,
-			exactlyOnce);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getJobCheckpointingSettings()).thenReturn(settings);
-
-		return new GraphAndSettings(graph, settings, externalizedSetting);
-	}
-
-	private static class GraphAndSettings {
-		public final AccessExecutionGraph graph;
-		public final JobCheckpointingSettings snapshottingSettings;
-		public final ExternalizedCheckpointSettings externalizedSettings;
-
-		public GraphAndSettings(
-				AccessExecutionGraph graph,
-				JobCheckpointingSettings snapshottingSettings,
-				ExternalizedCheckpointSettings externalizedSettings) {
-			this.graph = graph;
-			this.snapshottingSettings = snapshottingSettings;
-			this.externalizedSettings = externalizedSettings;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCacheTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCacheTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCacheTest.java
deleted file mode 100644
index bdb3faf..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsCacheTest.java
+++ /dev/null
@@ -1,71 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the CheckpoitnStatsCache.
- */
-public class CheckpointStatsCacheTest {
-
-	@Test
-	public void testZeroSizeCache() throws Exception {
-		AbstractCheckpointStats checkpoint = createCheckpoint(0, CheckpointStatsStatus.COMPLETED);
-
-		CheckpointStatsCache cache = new CheckpointStatsCache(0);
-		cache.tryAdd(checkpoint);
-		assertNull(cache.tryGet(0L));
-	}
-
-	@Test
-	public void testCacheAddAndGet() throws Exception {
-		AbstractCheckpointStats chk0 = createCheckpoint(0, CheckpointStatsStatus.COMPLETED);
-		AbstractCheckpointStats chk1 = createCheckpoint(1, CheckpointStatsStatus.COMPLETED);
-		AbstractCheckpointStats chk2 = createCheckpoint(2, CheckpointStatsStatus.IN_PROGRESS);
-
-		CheckpointStatsCache cache = new CheckpointStatsCache(1);
-		cache.tryAdd(chk0);
-		assertEquals(chk0, cache.tryGet(0));
-
-		cache.tryAdd(chk1);
-		assertNull(cache.tryGet(0));
-		assertEquals(chk1, cache.tryGet(1));
-
-		cache.tryAdd(chk2);
-		assertNull(cache.tryGet(2));
-		assertNull(cache.tryGet(0));
-		assertEquals(chk1, cache.tryGet(1));
-	}
-
-	private AbstractCheckpointStats createCheckpoint(long id, CheckpointStatsStatus status) {
-		AbstractCheckpointStats checkpoint = mock(AbstractCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(id);
-		when(checkpoint.getStatus()).thenReturn(status);
-		return checkpoint;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java
deleted file mode 100644
index f16d623..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java
+++ /dev/null
@@ -1,358 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointProperties;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
-import org.apache.flink.runtime.checkpoint.TaskStateStats;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the CheckpointStatsDetailsHandler.
- */
-public class CheckpointStatsDetailsHandlerTest {
-
-	@Test
-	public void testArchiver() throws IOException {
-		JsonArchivist archivist = new CheckpointStatsDetailsHandler.CheckpointStatsDetailsJsonArchivist();
-
-		CompletedCheckpointStats completedCheckpoint = createCompletedCheckpoint();
-		FailedCheckpointStats failedCheckpoint = createFailedCheckpoint();
-		List<AbstractCheckpointStats> checkpoints = new ArrayList<>();
-		checkpoints.add(failedCheckpoint);
-		checkpoints.add(completedCheckpoint);
-
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpoints()).thenReturn(checkpoints);
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-		when(graph.getJobID()).thenReturn(new JobID());
-
-		ObjectMapper mapper = new ObjectMapper();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(graph);
-		Assert.assertEquals(2, archives.size());
-
-		Iterator<ArchivedJson> iterator = archives.iterator();
-		ArchivedJson archive1 = iterator.next();
-		Assert.assertEquals(
-			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + failedCheckpoint.getCheckpointId(),
-			archive1.getPath());
-		compareFailedCheckpoint(failedCheckpoint, mapper.readTree(archive1.getJson()));
-
-		ArchivedJson archive2 = iterator.next();
-		Assert.assertEquals(
-			"/jobs/" + graph.getJobID() + "/checkpoints/details/" + completedCheckpoint.getCheckpointId(),
-			archive2.getPath());
-		compareCompletedCheckpoint(completedCheckpoint, mapper.readTree(archive2.getJson()));
-	}
-
-	@Test
-	public void testGetPaths() {
-		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid", paths[0]);
-	}
-
-	/**
-	 * Tests request with illegal checkpoint ID param.
-	 */
-	@Test
-	public void testIllegalCheckpointId() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "illegal checkpoint");
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Tests request with missing checkpoint ID param.
-	 */
-	@Test
-	public void testNoCheckpointIdParam() throws Exception {
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		String json = handler.handleRequest(graph, Collections.<String, String>emptyMap()).get();
-
-		assertEquals("{}", json);
-	}
-
-	/**
-	 * Test lookup of not existing checkpoint in history.
-	 */
-	@Test
-	public void testCheckpointNotFound() throws Exception {
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpointById(anyLong())).thenReturn(null); // not found
-
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "123");
-		String json = handler.handleRequest(graph, params).get();
-
-		assertEquals("{}", json);
-		verify(history, times(1)).getCheckpointById(anyLong());
-	}
-
-	/**
-	 * Tests a checkpoint details request for an in progress checkpoint.
-	 */
-	@Test
-	public void testCheckpointDetailsRequestInProgressCheckpoint() throws Exception {
-		PendingCheckpointStats checkpoint = mock(PendingCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1992139L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
-		when(checkpoint.getTriggerTimestamp()).thenReturn(1919191900L);
-		when(checkpoint.getLatestAckTimestamp()).thenReturn(1977791901L);
-		when(checkpoint.getStateSize()).thenReturn(111939272822L);
-		when(checkpoint.getEndToEndDuration()).thenReturn(121191L);
-		when(checkpoint.getAlignmentBuffered()).thenReturn(1L);
-		when(checkpoint.getNumberOfSubtasks()).thenReturn(501);
-		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(101);
-
-		List<TaskStateStats> taskStats = new ArrayList<>();
-		TaskStateStats task1 = createTaskStateStats();
-		TaskStateStats task2 = createTaskStateStats();
-		taskStats.add(task1);
-		taskStats.add(task2);
-
-		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
-
-		JsonNode rootNode = triggerRequest(checkpoint);
-
-		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
-		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
-		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
-		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
-		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
-		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
-		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
-		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
-		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
-		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
-
-		verifyTaskNodes(taskStats, rootNode);
-	}
-
-	/**
-	 * Tests a checkpoint details request for a completed checkpoint.
-	 */
-	@Test
-	public void testCheckpointDetailsRequestCompletedCheckpoint() throws Exception {
-		CompletedCheckpointStats checkpoint = createCompletedCheckpoint();
-
-		JsonNode rootNode = triggerRequest(checkpoint);
-
-		compareCompletedCheckpoint(checkpoint, rootNode);
-
-		verifyTaskNodes(checkpoint.getAllTaskStateStats(), rootNode);
-	}
-
-	/**
-	 * Tests a checkpoint details request for a failed checkpoint.
-	 */
-	@Test
-	public void testCheckpointDetailsRequestFailedCheckpoint() throws Exception {
-		FailedCheckpointStats checkpoint = createFailedCheckpoint();
-
-		JsonNode rootNode = triggerRequest(checkpoint);
-
-		compareFailedCheckpoint(checkpoint, rootNode);
-
-		verifyTaskNodes(checkpoint.getAllTaskStateStats(), rootNode);
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static CompletedCheckpointStats createCompletedCheckpoint() {
-		CompletedCheckpointStats checkpoint = mock(CompletedCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1818213L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
-		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
-		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
-		when(checkpoint.getStateSize()).thenReturn(925281L);
-		when(checkpoint.getEndToEndDuration()).thenReturn(181819L);
-		when(checkpoint.getAlignmentBuffered()).thenReturn(1010198L);
-		when(checkpoint.getNumberOfSubtasks()).thenReturn(181271);
-		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(29821);
-		when(checkpoint.isDiscarded()).thenReturn(true);
-		when(checkpoint.getExternalPath()).thenReturn("checkpoint-external-path");
-
-		List<TaskStateStats> taskStats = new ArrayList<>();
-		TaskStateStats task1 = createTaskStateStats();
-		TaskStateStats task2 = createTaskStateStats();
-		taskStats.add(task1);
-		taskStats.add(task2);
-
-		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
-
-		return checkpoint;
-	}
-
-	private static void compareCompletedCheckpoint(CompletedCheckpointStats checkpoint, JsonNode rootNode) {
-		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
-		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
-		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
-		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
-		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
-		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
-		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
-		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
-		assertEquals(checkpoint.isDiscarded(), rootNode.get("discarded").asBoolean());
-		assertEquals(checkpoint.getExternalPath(), rootNode.get("external_path").asText());
-		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
-		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
-	}
-
-	private static FailedCheckpointStats createFailedCheckpoint() {
-		FailedCheckpointStats checkpoint = mock(FailedCheckpointStats.class);
-		when(checkpoint.getCheckpointId()).thenReturn(1818214L);
-		when(checkpoint.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
-		when(checkpoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
-		when(checkpoint.getTriggerTimestamp()).thenReturn(1818L);
-		when(checkpoint.getLatestAckTimestamp()).thenReturn(11029222L);
-		when(checkpoint.getStateSize()).thenReturn(925281L);
-		when(checkpoint.getEndToEndDuration()).thenReturn(181819L);
-		when(checkpoint.getAlignmentBuffered()).thenReturn(1010198L);
-		when(checkpoint.getNumberOfSubtasks()).thenReturn(181271);
-		when(checkpoint.getNumberOfAcknowledgedSubtasks()).thenReturn(29821);
-		when(checkpoint.getFailureTimestamp()).thenReturn(123012890312093L);
-		when(checkpoint.getFailureMessage()).thenReturn("failure-message");
-
-		List<TaskStateStats> taskStats = new ArrayList<>();
-		TaskStateStats task1 = createTaskStateStats();
-		TaskStateStats task2 = createTaskStateStats();
-		taskStats.add(task1);
-		taskStats.add(task2);
-
-		when(checkpoint.getAllTaskStateStats()).thenReturn(taskStats);
-
-		return checkpoint;
-	}
-
-	private static void compareFailedCheckpoint(FailedCheckpointStats checkpoint, JsonNode rootNode) {
-		assertEquals(checkpoint.getCheckpointId(), rootNode.get("id").asLong());
-		assertEquals(checkpoint.getStatus().toString(), rootNode.get("status").asText());
-		assertEquals(checkpoint.getProperties().isSavepoint(), rootNode.get("is_savepoint").asBoolean());
-		assertEquals(checkpoint.getTriggerTimestamp(), rootNode.get("trigger_timestamp").asLong());
-		assertEquals(checkpoint.getLatestAckTimestamp(), rootNode.get("latest_ack_timestamp").asLong());
-		assertEquals(checkpoint.getStateSize(), rootNode.get("state_size").asLong());
-		assertEquals(checkpoint.getEndToEndDuration(), rootNode.get("end_to_end_duration").asLong());
-		assertEquals(checkpoint.getAlignmentBuffered(), rootNode.get("alignment_buffered").asLong());
-		assertEquals(checkpoint.getFailureTimestamp(), rootNode.get("failure_timestamp").asLong());
-		assertEquals(checkpoint.getFailureMessage(), rootNode.get("failure_message").asText());
-		assertEquals(checkpoint.getNumberOfSubtasks(), rootNode.get("num_subtasks").asInt());
-		assertEquals(checkpoint.getNumberOfAcknowledgedSubtasks(), rootNode.get("num_acknowledged_subtasks").asInt());
-	}
-
-	private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throws Exception {
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		when(history.getCheckpointById(anyLong())).thenReturn(checkpoint);
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getHistory()).thenReturn(history);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0));
-		Map<String, String> params = new HashMap<>();
-		params.put("checkpointid", "123");
-		String json = handler.handleRequest(graph, params).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		return mapper.readTree(json);
-	}
-
-	private static void verifyTaskNodes(Collection<TaskStateStats> tasks, JsonNode parentNode) {
-		for (TaskStateStats task : tasks) {
-			long duration = ThreadLocalRandom.current().nextInt(128);
-
-			JsonNode taskNode = parentNode.get("tasks").get(task.getJobVertexId().toString());
-			assertEquals(task.getLatestAckTimestamp(), taskNode.get("latest_ack_timestamp").asLong());
-			assertEquals(task.getStateSize(), taskNode.get("state_size").asLong());
-			assertEquals(task.getEndToEndDuration(task.getLatestAckTimestamp() - duration), taskNode.get("end_to_end_duration").asLong());
-			assertEquals(task.getAlignmentBuffered(), taskNode.get("alignment_buffered").asLong());
-			assertEquals(task.getNumberOfSubtasks(), taskNode.get("num_subtasks").asInt());
-			assertEquals(task.getNumberOfAcknowledgedSubtasks(), taskNode.get("num_acknowledged_subtasks").asInt());
-		}
-	}
-
-	private static TaskStateStats createTaskStateStats() {
-		ThreadLocalRandom rand = ThreadLocalRandom.current();
-
-		TaskStateStats task = mock(TaskStateStats.class);
-		when(task.getJobVertexId()).thenReturn(new JobVertexID());
-		when(task.getLatestAckTimestamp()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getStateSize()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getEndToEndDuration(anyLong())).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getAlignmentBuffered()).thenReturn(rand.nextLong(1024) + 1);
-		when(task.getNumberOfSubtasks()).thenReturn(rand.nextInt(1024) + 1);
-		when(task.getNumberOfAcknowledgedSubtasks()).thenReturn(rand.nextInt(1024) + 1);
-		return task;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java
deleted file mode 100644
index ed73a62..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java
+++ /dev/null
@@ -1,432 +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.webmonitor.handlers.checkpoints;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CheckpointProperties;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
-import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStatsSummary;
-import org.apache.flink.runtime.checkpoint.FailedCheckpointStats;
-import org.apache.flink.runtime.checkpoint.MinMaxAvgStats;
-import org.apache.flink.runtime.checkpoint.PendingCheckpointStats;
-import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the CheckpointStatsHandler.
- */
-public class CheckpointStatsHandlerTest {
-
-	@Test
-	public void testArchiver() throws IOException {
-		JsonArchivist archivist = new CheckpointStatsDetailsHandler.CheckpointStatsDetailsJsonArchivist();
-		TestCheckpointStats testCheckpointStats = createTestCheckpointStats();
-		when(testCheckpointStats.graph.getJobID()).thenReturn(new JobID());
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(testCheckpointStats.graph);
-		Assert.assertEquals(3, archives.size());
-
-		ObjectMapper mapper = new ObjectMapper();
-
-		Iterator<ArchivedJson> iterator = archives.iterator();
-		ArchivedJson archive1 = iterator.next();
-		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.inProgress.getCheckpointId(), archive1.getPath());
-		compareInProgressCheckpoint(testCheckpointStats.inProgress, mapper.readTree(archive1.getJson()));
-
-		ArchivedJson archive2 = iterator.next();
-		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.completedSavepoint.getCheckpointId(), archive2.getPath());
-		compareCompletedSavepoint(testCheckpointStats.completedSavepoint, mapper.readTree(archive2.getJson()));
-
-		ArchivedJson archive3 = iterator.next();
-		Assert.assertEquals("/jobs/" + testCheckpointStats.graph.getJobID() + "/checkpoints/details/" + testCheckpointStats.failed.getCheckpointId(), archive3.getPath());
-		compareFailedCheckpoint(testCheckpointStats.failed, mapper.readTree(archive3.getJson()));
-	}
-
-	@Test
-	public void testGetPaths() {
-		CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/checkpoints", paths[0]);
-	}
-
-	/**
-	 * Tests a complete checkpoint stats snapshot.
-	 */
-	@Test
-	public void testCheckpointStatsRequest() throws Exception {
-		TestCheckpointStats testCheckpointStats = createTestCheckpointStats();
-
-		CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String json = handler.handleRequest(testCheckpointStats.graph, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(json);
-
-		compareCheckpointStats(testCheckpointStats, rootNode);
-	}
-
-	private static TestCheckpointStats createTestCheckpointStats() {
-		// Counts
-		CheckpointStatsCounts counts = mock(CheckpointStatsCounts.class);
-		when(counts.getNumberOfRestoredCheckpoints()).thenReturn(123123123L);
-		when(counts.getTotalNumberOfCheckpoints()).thenReturn(12981231203L);
-		when(counts.getNumberOfInProgressCheckpoints()).thenReturn(191919);
-		when(counts.getNumberOfCompletedCheckpoints()).thenReturn(882828200L);
-		when(counts.getNumberOfFailedCheckpoints()).thenReturn(99171510L);
-
-		// Summary
-		CompletedCheckpointStatsSummary summary = mock(CompletedCheckpointStatsSummary.class);
-
-		MinMaxAvgStats stateSizeSummary = mock(MinMaxAvgStats.class);
-		when(stateSizeSummary.getMinimum()).thenReturn(81238123L);
-		when(stateSizeSummary.getMaximum()).thenReturn(19919191999L);
-		when(stateSizeSummary.getAverage()).thenReturn(1133L);
-
-		MinMaxAvgStats durationSummary = mock(MinMaxAvgStats.class);
-		when(durationSummary.getMinimum()).thenReturn(1182L);
-		when(durationSummary.getMaximum()).thenReturn(88654L);
-		when(durationSummary.getAverage()).thenReturn(171L);
-
-		MinMaxAvgStats alignmentBufferedSummary = mock(MinMaxAvgStats.class);
-		when(alignmentBufferedSummary.getMinimum()).thenReturn(81818181899L);
-		when(alignmentBufferedSummary.getMaximum()).thenReturn(89999911118654L);
-		when(alignmentBufferedSummary.getAverage()).thenReturn(11203131L);
-
-		when(summary.getStateSizeStats()).thenReturn(stateSizeSummary);
-		when(summary.getEndToEndDurationStats()).thenReturn(durationSummary);
-		when(summary.getAlignmentBufferedStats()).thenReturn(alignmentBufferedSummary);
-
-		// Latest
-		CompletedCheckpointStats latestCompleted = mock(CompletedCheckpointStats.class);
-		when(latestCompleted.getCheckpointId()).thenReturn(1992139L);
-		when(latestCompleted.getTriggerTimestamp()).thenReturn(1919191900L);
-		when(latestCompleted.getLatestAckTimestamp()).thenReturn(1977791901L);
-		when(latestCompleted.getStateSize()).thenReturn(111939272822L);
-		when(latestCompleted.getEndToEndDuration()).thenReturn(121191L);
-		when(latestCompleted.getAlignmentBuffered()).thenReturn(1L);
-		when(latestCompleted.getExternalPath()).thenReturn("latest-completed-external-path");
-
-		CompletedCheckpointStats latestSavepoint = mock(CompletedCheckpointStats.class);
-		when(latestSavepoint.getCheckpointId()).thenReturn(1992140L);
-		when(latestSavepoint.getTriggerTimestamp()).thenReturn(1919191900L);
-		when(latestSavepoint.getLatestAckTimestamp()).thenReturn(1977791901L);
-		when(latestSavepoint.getStateSize()).thenReturn(111939272822L);
-		when(latestSavepoint.getEndToEndDuration()).thenReturn(121191L);
-		when(latestCompleted.getAlignmentBuffered()).thenReturn(182813L);
-		when(latestSavepoint.getExternalPath()).thenReturn("savepoint-external-path");
-
-		FailedCheckpointStats latestFailed = mock(FailedCheckpointStats.class);
-		when(latestFailed.getCheckpointId()).thenReturn(1112L);
-		when(latestFailed.getTriggerTimestamp()).thenReturn(12828L);
-		when(latestFailed.getLatestAckTimestamp()).thenReturn(1901L);
-		when(latestFailed.getFailureTimestamp()).thenReturn(11999976L);
-		when(latestFailed.getStateSize()).thenReturn(111L);
-		when(latestFailed.getEndToEndDuration()).thenReturn(12L);
-		when(latestFailed.getAlignmentBuffered()).thenReturn(2L);
-		when(latestFailed.getFailureMessage()).thenReturn("expected cause");
-
-		RestoredCheckpointStats latestRestored = mock(RestoredCheckpointStats.class);
-		when(latestRestored.getCheckpointId()).thenReturn(1199L);
-		when(latestRestored.getRestoreTimestamp()).thenReturn(434242L);
-		when(latestRestored.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
-		when(latestRestored.getExternalPath()).thenReturn("restored savepoint path");
-
-		// History
-		CheckpointStatsHistory history = mock(CheckpointStatsHistory.class);
-		List<AbstractCheckpointStats> checkpoints = new ArrayList<>();
-
-		PendingCheckpointStats inProgress = mock(PendingCheckpointStats.class);
-		when(inProgress.getCheckpointId()).thenReturn(1992141L);
-		when(inProgress.getStatus()).thenReturn(CheckpointStatsStatus.IN_PROGRESS);
-		when(inProgress.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
-		when(inProgress.getTriggerTimestamp()).thenReturn(1919191900L);
-		when(inProgress.getLatestAckTimestamp()).thenReturn(1977791901L);
-		when(inProgress.getStateSize()).thenReturn(111939272822L);
-		when(inProgress.getEndToEndDuration()).thenReturn(121191L);
-		when(inProgress.getAlignmentBuffered()).thenReturn(1L);
-		when(inProgress.getNumberOfSubtasks()).thenReturn(501);
-		when(inProgress.getNumberOfAcknowledgedSubtasks()).thenReturn(101);
-
-		CompletedCheckpointStats completedSavepoint = mock(CompletedCheckpointStats.class);
-		when(completedSavepoint.getCheckpointId()).thenReturn(1322139L);
-		when(completedSavepoint.getStatus()).thenReturn(CheckpointStatsStatus.COMPLETED);
-		when(completedSavepoint.getProperties()).thenReturn(CheckpointProperties.forStandardSavepoint());
-		when(completedSavepoint.getTriggerTimestamp()).thenReturn(191900L);
-		when(completedSavepoint.getLatestAckTimestamp()).thenReturn(197791901L);
-		when(completedSavepoint.getStateSize()).thenReturn(1119822L);
-		when(completedSavepoint.getEndToEndDuration()).thenReturn(12191L);
-		when(completedSavepoint.getAlignmentBuffered()).thenReturn(111L);
-		when(completedSavepoint.getNumberOfSubtasks()).thenReturn(33501);
-		when(completedSavepoint.getNumberOfAcknowledgedSubtasks()).thenReturn(211);
-		when(completedSavepoint.isDiscarded()).thenReturn(true);
-		when(completedSavepoint.getExternalPath()).thenReturn("completed-external-path");
-
-		FailedCheckpointStats failed = mock(FailedCheckpointStats.class);
-		when(failed.getCheckpointId()).thenReturn(110719L);
-		when(failed.getStatus()).thenReturn(CheckpointStatsStatus.FAILED);
-		when(failed.getProperties()).thenReturn(CheckpointProperties.forStandardCheckpoint());
-		when(failed.getTriggerTimestamp()).thenReturn(191900L);
-		when(failed.getLatestAckTimestamp()).thenReturn(197791901L);
-		when(failed.getStateSize()).thenReturn(1119822L);
-		when(failed.getEndToEndDuration()).thenReturn(12191L);
-		when(failed.getAlignmentBuffered()).thenReturn(111L);
-		when(failed.getNumberOfSubtasks()).thenReturn(33501);
-		when(failed.getNumberOfAcknowledgedSubtasks()).thenReturn(1);
-		when(failed.getFailureTimestamp()).thenReturn(119230L);
-		when(failed.getFailureMessage()).thenReturn("failure message");
-
-		checkpoints.add(inProgress);
-		checkpoints.add(completedSavepoint);
-		checkpoints.add(failed);
-		when(history.getCheckpoints()).thenReturn(checkpoints);
-		when(history.getLatestCompletedCheckpoint()).thenReturn(latestCompleted);
-		when(history.getLatestSavepoint()).thenReturn(latestSavepoint);
-		when(history.getLatestFailedCheckpoint()).thenReturn(latestFailed);
-
-		CheckpointStatsSnapshot snapshot = mock(CheckpointStatsSnapshot.class);
-		when(snapshot.getCounts()).thenReturn(counts);
-		when(snapshot.getSummaryStats()).thenReturn(summary);
-		when(snapshot.getHistory()).thenReturn(history);
-		when(snapshot.getLatestRestoredCheckpoint()).thenReturn(latestRestored);
-
-		AccessExecutionGraph graph = mock(AccessExecutionGraph.class);
-		when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot);
-
-		return new TestCheckpointStats(
-			graph, counts, stateSizeSummary, durationSummary, alignmentBufferedSummary, summary,
-			latestCompleted, latestSavepoint, latestFailed, latestRestored, inProgress,
-			completedSavepoint, failed, history, snapshot
-		);
-	}
-
-	private static void compareCheckpointStats(TestCheckpointStats checkpointStats, JsonNode rootNode) {
-		CheckpointStatsCounts counts = checkpointStats.counts;
-		JsonNode countNode = rootNode.get("counts");
-		assertEquals(counts.getNumberOfRestoredCheckpoints(), countNode.get("restored").asLong());
-		assertEquals(counts.getTotalNumberOfCheckpoints(), countNode.get("total").asLong());
-		assertEquals(counts.getNumberOfInProgressCheckpoints(), countNode.get("in_progress").asLong());
-		assertEquals(counts.getNumberOfCompletedCheckpoints(), countNode.get("completed").asLong());
-		assertEquals(counts.getNumberOfFailedCheckpoints(), countNode.get("failed").asLong());
-
-		MinMaxAvgStats stateSizeSummary = checkpointStats.stateSizeSummary;
-		JsonNode summaryNode = rootNode.get("summary");
-		JsonNode sizeSummaryNode = summaryNode.get("state_size");
-		assertEquals(stateSizeSummary.getMinimum(), sizeSummaryNode.get("min").asLong());
-		assertEquals(stateSizeSummary.getMaximum(), sizeSummaryNode.get("max").asLong());
-		assertEquals(stateSizeSummary.getAverage(), sizeSummaryNode.get("avg").asLong());
-
-		MinMaxAvgStats durationSummary = checkpointStats.durationSummary;
-		JsonNode durationSummaryNode = summaryNode.get("end_to_end_duration");
-		assertEquals(durationSummary.getMinimum(), durationSummaryNode.get("min").asLong());
-		assertEquals(durationSummary.getMaximum(), durationSummaryNode.get("max").asLong());
-		assertEquals(durationSummary.getAverage(), durationSummaryNode.get("avg").asLong());
-
-		MinMaxAvgStats alignmentBufferedSummary = checkpointStats.alignmentBufferedSummary;
-		JsonNode alignmentBufferedNode = summaryNode.get("alignment_buffered");
-		assertEquals(alignmentBufferedSummary.getMinimum(), alignmentBufferedNode.get("min").asLong());
-		assertEquals(alignmentBufferedSummary.getMaximum(), alignmentBufferedNode.get("max").asLong());
-		assertEquals(alignmentBufferedSummary.getAverage(), alignmentBufferedNode.get("avg").asLong());
-
-		CompletedCheckpointStats latestCompleted = checkpointStats.latestCompleted;
-		JsonNode latestNode = rootNode.get("latest");
-		JsonNode latestCheckpointNode = latestNode.get("completed");
-		assertEquals(latestCompleted.getCheckpointId(), latestCheckpointNode.get("id").asLong());
-		assertEquals(latestCompleted.getTriggerTimestamp(), latestCheckpointNode.get("trigger_timestamp").asLong());
-		assertEquals(latestCompleted.getLatestAckTimestamp(), latestCheckpointNode.get("latest_ack_timestamp").asLong());
-		assertEquals(latestCompleted.getStateSize(), latestCheckpointNode.get("state_size").asLong());
-		assertEquals(latestCompleted.getEndToEndDuration(), latestCheckpointNode.get("end_to_end_duration").asLong());
-		assertEquals(latestCompleted.getAlignmentBuffered(), latestCheckpointNode.get("alignment_buffered").asLong());
-		assertEquals(latestCompleted.getExternalPath(), latestCheckpointNode.get("external_path").asText());
-
-		CompletedCheckpointStats latestSavepoint = checkpointStats.latestSavepoint;
-		JsonNode latestSavepointNode = latestNode.get("savepoint");
-		assertEquals(latestSavepoint.getCheckpointId(), latestSavepointNode.get("id").asLong());
-		assertEquals(latestSavepoint.getTriggerTimestamp(), latestSavepointNode.get("trigger_timestamp").asLong());
-		assertEquals(latestSavepoint.getLatestAckTimestamp(), latestSavepointNode.get("latest_ack_timestamp").asLong());
-		assertEquals(latestSavepoint.getStateSize(), latestSavepointNode.get("state_size").asLong());
-		assertEquals(latestSavepoint.getEndToEndDuration(), latestSavepointNode.get("end_to_end_duration").asLong());
-		assertEquals(latestSavepoint.getAlignmentBuffered(), latestSavepointNode.get("alignment_buffered").asLong());
-		assertEquals(latestSavepoint.getExternalPath(), latestSavepointNode.get("external_path").asText());
-
-		FailedCheckpointStats latestFailed = checkpointStats.latestFailed;
-		JsonNode latestFailedNode = latestNode.get("failed");
-		assertEquals(latestFailed.getCheckpointId(), latestFailedNode.get("id").asLong());
-		assertEquals(latestFailed.getTriggerTimestamp(), latestFailedNode.get("trigger_timestamp").asLong());
-		assertEquals(latestFailed.getLatestAckTimestamp(), latestFailedNode.get("latest_ack_timestamp").asLong());
-		assertEquals(latestFailed.getStateSize(), latestFailedNode.get("state_size").asLong());
-		assertEquals(latestFailed.getEndToEndDuration(), latestFailedNode.get("end_to_end_duration").asLong());
-		assertEquals(latestFailed.getAlignmentBuffered(), latestFailedNode.get("alignment_buffered").asLong());
-		assertEquals(latestFailed.getFailureTimestamp(), latestFailedNode.get("failure_timestamp").asLong());
-		assertEquals(latestFailed.getFailureMessage(), latestFailedNode.get("failure_message").asText());
-
-		RestoredCheckpointStats latestRestored = checkpointStats.latestRestored;
-		JsonNode latestRestoredNode = latestNode.get("restored");
-		assertEquals(latestRestored.getCheckpointId(), latestRestoredNode.get("id").asLong());
-		assertEquals(latestRestored.getRestoreTimestamp(), latestRestoredNode.get("restore_timestamp").asLong());
-		assertEquals(latestRestored.getProperties().isSavepoint(), latestRestoredNode.get("is_savepoint").asBoolean());
-		assertEquals(latestRestored.getExternalPath(), latestRestoredNode.get("external_path").asText());
-
-		JsonNode historyNode = rootNode.get("history");
-		Iterator<JsonNode> it = historyNode.iterator();
-
-		assertTrue(it.hasNext());
-		JsonNode inProgressNode = it.next();
-
-		PendingCheckpointStats inProgress = checkpointStats.inProgress;
-		compareInProgressCheckpoint(inProgress, inProgressNode);
-
-		assertTrue(it.hasNext());
-		JsonNode completedSavepointNode = it.next();
-
-		CompletedCheckpointStats completedSavepoint = checkpointStats.completedSavepoint;
-		compareCompletedSavepoint(completedSavepoint, completedSavepointNode);
-
-		assertTrue(it.hasNext());
-		JsonNode failedNode = it.next();
-
-		FailedCheckpointStats failed = checkpointStats.failed;
-		compareFailedCheckpoint(failed, failedNode);
-
-		assertFalse(it.hasNext());
-	}
-
-	private static void compareInProgressCheckpoint(PendingCheckpointStats inProgress, JsonNode inProgressNode) {
-		assertEquals(inProgress.getCheckpointId(), inProgressNode.get("id").asLong());
-		assertEquals(inProgress.getStatus().toString(), inProgressNode.get("status").asText());
-		assertEquals(inProgress.getProperties().isSavepoint(), inProgressNode.get("is_savepoint").asBoolean());
-		assertEquals(inProgress.getTriggerTimestamp(), inProgressNode.get("trigger_timestamp").asLong());
-		assertEquals(inProgress.getLatestAckTimestamp(), inProgressNode.get("latest_ack_timestamp").asLong());
-		assertEquals(inProgress.getStateSize(), inProgressNode.get("state_size").asLong());
-		assertEquals(inProgress.getEndToEndDuration(), inProgressNode.get("end_to_end_duration").asLong());
-		assertEquals(inProgress.getAlignmentBuffered(), inProgressNode.get("alignment_buffered").asLong());
-		assertEquals(inProgress.getNumberOfSubtasks(), inProgressNode.get("num_subtasks").asInt());
-		assertEquals(inProgress.getNumberOfAcknowledgedSubtasks(), inProgressNode.get("num_acknowledged_subtasks").asInt());
-	}
-
-	private static void compareCompletedSavepoint(CompletedCheckpointStats completedSavepoint, JsonNode completedSavepointNode) {
-		assertEquals(completedSavepoint.getCheckpointId(), completedSavepointNode.get("id").asLong());
-		assertEquals(completedSavepoint.getStatus().toString(), completedSavepointNode.get("status").asText());
-		assertEquals(completedSavepoint.getProperties().isSavepoint(), completedSavepointNode.get("is_savepoint").asBoolean());
-		assertEquals(completedSavepoint.getTriggerTimestamp(), completedSavepointNode.get("trigger_timestamp").asLong());
-		assertEquals(completedSavepoint.getLatestAckTimestamp(), completedSavepointNode.get("latest_ack_timestamp").asLong());
-		assertEquals(completedSavepoint.getStateSize(), completedSavepointNode.get("state_size").asLong());
-		assertEquals(completedSavepoint.getEndToEndDuration(), completedSavepointNode.get("end_to_end_duration").asLong());
-		assertEquals(completedSavepoint.getAlignmentBuffered(), completedSavepointNode.get("alignment_buffered").asLong());
-		assertEquals(completedSavepoint.getNumberOfSubtasks(), completedSavepointNode.get("num_subtasks").asInt());
-		assertEquals(completedSavepoint.getNumberOfAcknowledgedSubtasks(), completedSavepointNode.get("num_acknowledged_subtasks").asInt());
-
-		assertEquals(completedSavepoint.getExternalPath(), completedSavepointNode.get("external_path").asText());
-		assertEquals(completedSavepoint.isDiscarded(), completedSavepointNode.get("discarded").asBoolean());
-	}
-
-	private static void compareFailedCheckpoint(FailedCheckpointStats failed, JsonNode failedNode) {
-		assertEquals(failed.getCheckpointId(), failedNode.get("id").asLong());
-		assertEquals(failed.getStatus().toString(), failedNode.get("status").asText());
-		assertEquals(failed.getProperties().isSavepoint(), failedNode.get("is_savepoint").asBoolean());
-		assertEquals(failed.getTriggerTimestamp(), failedNode.get("trigger_timestamp").asLong());
-		assertEquals(failed.getLatestAckTimestamp(), failedNode.get("latest_ack_timestamp").asLong());
-		assertEquals(failed.getStateSize(), failedNode.get("state_size").asLong());
-		assertEquals(failed.getEndToEndDuration(), failedNode.get("end_to_end_duration").asLong());
-		assertEquals(failed.getAlignmentBuffered(), failedNode.get("alignment_buffered").asLong());
-		assertEquals(failed.getNumberOfSubtasks(), failedNode.get("num_subtasks").asInt());
-		assertEquals(failed.getNumberOfAcknowledgedSubtasks(), failedNode.get("num_acknowledged_subtasks").asInt());
-
-		assertEquals(failed.getFailureTimestamp(), failedNode.get("failure_timestamp").asLong());
-		assertEquals(failed.getFailureMessage(), failedNode.get("failure_message").asText());
-	}
-
-	private static class TestCheckpointStats {
-		public final AccessExecutionGraph graph;
-		public final CheckpointStatsCounts counts;
-		public final MinMaxAvgStats stateSizeSummary;
-		public final MinMaxAvgStats durationSummary;
-		public final MinMaxAvgStats alignmentBufferedSummary;
-		public final CompletedCheckpointStatsSummary summary;
-		public final CompletedCheckpointStats latestCompleted;
-		public final CompletedCheckpointStats latestSavepoint;
-		public final FailedCheckpointStats latestFailed;
-		public final RestoredCheckpointStats latestRestored;
-		public final PendingCheckpointStats inProgress;
-		public final CompletedCheckpointStats completedSavepoint;
-		public final FailedCheckpointStats failed;
-		public final CheckpointStatsHistory history;
-		public final CheckpointStatsSnapshot snapshot;
-
-		public TestCheckpointStats(
-				AccessExecutionGraph graph,
-				CheckpointStatsCounts counts,
-				MinMaxAvgStats stateSizeSummary,
-				MinMaxAvgStats durationSummary,
-				MinMaxAvgStats alignmentBufferedSummary,
-				CompletedCheckpointStatsSummary summary,
-				CompletedCheckpointStats latestCompleted,
-				CompletedCheckpointStats latestSavepoint,
-				FailedCheckpointStats latestFailed,
-				RestoredCheckpointStats latestRestored,
-				PendingCheckpointStats inProgress,
-				CompletedCheckpointStats completedSavepoint,
-				FailedCheckpointStats failed,
-				CheckpointStatsHistory history,
-				CheckpointStatsSnapshot snapshot) {
-			this.graph = graph;
-			this.counts = counts;
-			this.stateSizeSummary = stateSizeSummary;
-			this.durationSummary = durationSummary;
-			this.alignmentBufferedSummary = alignmentBufferedSummary;
-			this.summary = summary;
-			this.latestCompleted = latestCompleted;
-			this.latestSavepoint = latestSavepoint;
-			this.latestFailed = latestFailed;
-			this.latestRestored = latestRestored;
-			this.inProgress = inProgress;
-			this.completedSavepoint = completedSavepoint;
-			this.failed = failed;
-			this.history = history;
-			this.snapshot = snapshot;
-		}
-	}
-}


[11/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
deleted file mode 100644
index e34631e..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
+++ /dev/null
@@ -1,334 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the JobCancellationWithSavepointHandler.
- */
-public class JobCancellationWithSavepointHandlersTest extends TestLogger {
-
-	private static final Executor executor = Executors.directExecutor();
-
-	@Test
-	public void testGetPaths() {
-		JobCancellationWithSavepointHandlers handler = new JobCancellationWithSavepointHandlers(mock(ExecutionGraphHolder.class), executor);
-
-		JobCancellationWithSavepointHandlers.TriggerHandler triggerHandler = handler.getTriggerHandler();
-		String[] triggerPaths = triggerHandler.getPaths();
-		Assert.assertEquals(2, triggerPaths.length);
-		List<String> triggerPathsList = Arrays.asList(triggerPaths);
-		Assert.assertTrue(triggerPathsList.contains("/jobs/:jobid/cancel-with-savepoint"));
-		Assert.assertTrue(triggerPathsList.contains("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory"));
-
-		JobCancellationWithSavepointHandlers.InProgressHandler progressHandler = handler.getInProgressHandler();
-		String[] progressPaths = progressHandler.getPaths();
-		Assert.assertEquals(1, progressPaths.length);
-		Assert.assertEquals("/jobs/:jobid/cancel-with-savepoint/in-progress/:requestId", progressPaths[0]);
-	}
-
-	/**
-	 * Tests that the cancellation ask timeout respects the checkpoint timeout.
-	 * Otherwise, AskTimeoutExceptions are bound to happen for large state.
-	 */
-	@Test
-	public void testAskTimeoutEqualsCheckpointTimeout() throws Exception {
-		long timeout = 128288238L;
-		JobID jobId = new JobID();
-		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
-		ExecutionGraph graph = mock(ExecutionGraph.class);
-		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
-		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
-		when(graph.getCheckpointCoordinator()).thenReturn(coord);
-		when(coord.getCheckpointTimeout()).thenReturn(timeout);
-
-		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
-		JobCancellationWithSavepointHandlers.TriggerHandler handler = handlers.getTriggerHandler();
-
-		Map<String, String> params = new HashMap<>();
-		params.put("jobid", jobId.toString());
-		params.put("targetDirectory", "placeholder");
-
-		JobManagerGateway jobManager = mock(JobManagerGateway.class);
-		when(jobManager.cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class))).thenReturn(CompletableFuture.completedFuture("foobar"));
-
-		handler.handleRequest(params, Collections.emptyMap(), jobManager);
-
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class));
-	}
-
-	/**
-	 * Tests that the savepoint directory configuration is respected.
-	 */
-	@Test
-	public void testSavepointDirectoryConfiguration() throws Exception {
-		long timeout = 128288238L;
-		JobID jobId = new JobID();
-		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
-		ExecutionGraph graph = mock(ExecutionGraph.class);
-		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
-		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
-		when(graph.getCheckpointCoordinator()).thenReturn(coord);
-		when(coord.getCheckpointTimeout()).thenReturn(timeout);
-
-		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor, "the-default-directory");
-		JobCancellationWithSavepointHandlers.TriggerHandler handler = handlers.getTriggerHandler();
-
-		Map<String, String> params = new HashMap<>();
-		params.put("jobid", jobId.toString());
-
-		JobManagerGateway jobManager = mock(JobManagerGateway.class);
-		when(jobManager.cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class))).thenReturn(CompletableFuture.completedFuture("foobar"));
-
-		// 1. Use targetDirectory path param
-		params.put("targetDirectory", "custom-directory");
-		handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
-
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
-
-		// 2. Use default
-		params.remove("targetDirectory");
-
-		handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
-
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("the-default-directory"), any(Time.class));
-
-		// 3. Throw Exception
-		handlers = new JobCancellationWithSavepointHandlers(holder, executor, null);
-		handler = handlers.getTriggerHandler();
-
-		try {
-			handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-			fail("Did not throw expected test Exception");
-		} catch (Exception e) {
-			IllegalStateException cause = (IllegalStateException) e.getCause();
-			assertEquals(true, cause.getMessage().contains(CoreOptions.SAVEPOINT_DIRECTORY.key()));
-		}
-	}
-
-	/**
-	 * Tests triggering a new request and monitoring it.
-	 */
-	@Test
-	public void testTriggerNewRequest() throws Exception {
-		JobID jobId = new JobID();
-		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
-		ExecutionGraph graph = mock(ExecutionGraph.class);
-		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
-		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
-		when(graph.getCheckpointCoordinator()).thenReturn(coord);
-
-		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
-		JobCancellationWithSavepointHandlers.TriggerHandler trigger = handlers.getTriggerHandler();
-		JobCancellationWithSavepointHandlers.InProgressHandler progress = handlers.getInProgressHandler();
-
-		Map<String, String> params = new HashMap<>();
-		params.put("jobid", jobId.toString());
-		params.put("targetDirectory", "custom-directory");
-
-		JobManagerGateway jobManager = mock(JobManagerGateway.class);
-
-		// Successful
-		CompletableFuture<String> successfulCancelWithSavepoint = new CompletableFuture<>();
-		when(jobManager.cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class))).thenReturn(successfulCancelWithSavepoint);
-
-		// Trigger
-		FullHttpResponse response = trigger.handleRequest(params, Collections.emptyMap(), jobManager).get();
-
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
-
-		String location = String.format("/jobs/%s/cancel-with-savepoint/in-progress/1", jobId);
-
-		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-		assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
-
-		String json = response.content().toString(Charset.forName("UTF-8"));
-		JsonNode root = new ObjectMapper().readTree(json);
-
-		assertEquals("accepted", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-		assertEquals(location, root.get("location").asText());
-
-		// Trigger again
-		response = trigger.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-		assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
-
-		json = response.content().toString(Charset.forName("UTF-8"));
-		root = new ObjectMapper().readTree(json);
-
-		assertEquals("accepted", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-		assertEquals(location, root.get("location").asText());
-
-		// Only single actual request
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
-
-		// Query progress
-		params.put("requestId", "1");
-
-		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-
-		json = response.content().toString(Charset.forName("UTF-8"));
-		root = new ObjectMapper().readTree(json);
-
-		assertEquals("in-progress", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-
-		// Complete
-		successfulCancelWithSavepoint.complete("_path-savepoint_");
-
-		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-
-		assertEquals(HttpResponseStatus.CREATED, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-
-		json = response.content().toString(Charset.forName("UTF-8"));
-
-		root = new ObjectMapper().readTree(json);
-
-		assertEquals("success", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-		assertEquals("_path-savepoint_", root.get("savepoint-path").asText());
-
-		// Query again, keep recent history
-
-		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-
-		assertEquals(HttpResponseStatus.CREATED, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-
-		json = response.content().toString(Charset.forName("UTF-8"));
-
-		root = new ObjectMapper().readTree(json);
-
-		assertEquals("success", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-		assertEquals("_path-savepoint_", root.get("savepoint-path").asText());
-
-		// Query for unknown request
-		params.put("requestId", "9929");
-
-		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-		assertEquals(HttpResponseStatus.BAD_REQUEST, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-
-		json = response.content().toString(Charset.forName("UTF-8"));
-
-		root = new ObjectMapper().readTree(json);
-
-		assertEquals("failed", root.get("status").asText());
-		assertEquals("9929", root.get("request-id").asText());
-		assertEquals("Unknown job/request ID", root.get("cause").asText());
-	}
-
-	/**
-	 * Tests response when a request fails.
-	 */
-	@Test
-	public void testFailedCancellation() throws Exception {
-		JobID jobId = new JobID();
-		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
-		ExecutionGraph graph = mock(ExecutionGraph.class);
-		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
-		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
-		when(graph.getCheckpointCoordinator()).thenReturn(coord);
-
-		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
-		JobCancellationWithSavepointHandlers.TriggerHandler trigger = handlers.getTriggerHandler();
-		JobCancellationWithSavepointHandlers.InProgressHandler progress = handlers.getInProgressHandler();
-
-		Map<String, String> params = new HashMap<>();
-		params.put("jobid", jobId.toString());
-		params.put("targetDirectory", "custom-directory");
-
-		JobManagerGateway jobManager = mock(JobManagerGateway.class);
-
-		// Successful
-		CompletableFuture<String> unsuccessfulCancelWithSavepoint = FutureUtils.completedExceptionally(new Exception("Test Exception"));
-		when(jobManager.cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class))).thenReturn(unsuccessfulCancelWithSavepoint);
-
-		// Trigger
-		trigger.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
-		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
-
-		// Query progress
-		params.put("requestId", "1");
-
-		FullHttpResponse response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
-		assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR, response.getStatus());
-		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
-		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
-
-		String json = response.content().toString(Charset.forName("UTF-8"));
-		JsonNode root = new ObjectMapper().readTree(json);
-
-		assertEquals("failed", root.get("status").asText());
-		assertEquals("1", root.get("request-id").asText());
-		assertEquals("Test Exception", root.get("cause").asText());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java
deleted file mode 100644
index 1c08ae8..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java
+++ /dev/null
@@ -1,92 +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.webmonitor.handlers;
-
-import org.apache.flink.api.common.ArchivedExecutionConfig;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobConfigHandler.
- */
-public class JobConfigHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobConfigHandler.JobConfigJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/config", archive.getPath());
-		compareJobConfig(originalJob, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobConfigHandler handler = new JobConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/config", paths[0]);
-	}
-
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		String answer = JobConfigHandler.createJobConfigJson(originalJob);
-		compareJobConfig(originalJob, answer);
-	}
-
-	private static void compareJobConfig(AccessExecutionGraph originalJob, String answer) throws IOException {
-		JsonNode job = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
-
-		Assert.assertEquals(originalJob.getJobID().toString(), job.get("jid").asText());
-		Assert.assertEquals(originalJob.getJobName(), job.get("name").asText());
-
-		ArchivedExecutionConfig originalConfig = originalJob.getArchivedExecutionConfig();
-		JsonNode config = job.get("execution-config");
-
-		Assert.assertEquals(originalConfig.getExecutionMode(), config.get("execution-mode").asText());
-		Assert.assertEquals(originalConfig.getRestartStrategyDescription(), config.get("restart-strategy").asText());
-		Assert.assertEquals(originalConfig.getParallelism(), config.get("job-parallelism").asInt());
-		Assert.assertEquals(originalConfig.getObjectReuseEnabled(), config.get("object-reuse-mode").asBoolean());
-
-		Map<String, String> originalUserConfig = originalConfig.getGlobalJobParameters();
-		JsonNode userConfig = config.get("user-config");
-
-		for (Map.Entry<String, String> originalEntry : originalUserConfig.entrySet()) {
-			Assert.assertEquals(originalEntry.getValue(), userConfig.get(originalEntry.getKey()).asText());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java
deleted file mode 100644
index ee0498e..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java
+++ /dev/null
@@ -1,169 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.IOMetrics;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobDetailsHandler.
- */
-public class JobDetailsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobDetailsHandler.JobDetailsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(2, archives.size());
-
-		Iterator<ArchivedJson> iterator = archives.iterator();
-		ArchivedJson archive1 = iterator.next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID(), archive1.getPath());
-		compareJobDetails(originalJob, archive1.getJson());
-
-		ArchivedJson archive2 = iterator.next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices", archive2.getPath());
-		compareJobDetails(originalJob, archive2.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobDetailsHandler handler = new JobDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(2, paths.length);
-		List<String> pathsList = Lists.newArrayList(paths);
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid"));
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid/vertices"));
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		String json = JobDetailsHandler.createJobDetailsJson(originalJob, null);
-
-		compareJobDetails(originalJob, json);
-	}
-
-	private static void compareJobDetails(AccessExecutionGraph originalJob, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalJob.getJobID().toString(), result.get("jid").asText());
-		Assert.assertEquals(originalJob.getJobName(), result.get("name").asText());
-		Assert.assertEquals(originalJob.isStoppable(), result.get("isStoppable").asBoolean());
-		Assert.assertEquals(originalJob.getState().name(), result.get("state").asText());
-
-		Assert.assertEquals(originalJob.getStatusTimestamp(JobStatus.CREATED), result.get("start-time").asLong());
-		Assert.assertEquals(originalJob.getStatusTimestamp(originalJob.getState()), result.get("end-time").asLong());
-		Assert.assertEquals(
-			originalJob.getStatusTimestamp(originalJob.getState()) - originalJob.getStatusTimestamp(JobStatus.CREATED),
-			result.get("duration").asLong()
-		);
-
-		JsonNode timestamps = result.get("timestamps");
-		for (JobStatus status : JobStatus.values()) {
-			Assert.assertEquals(originalJob.getStatusTimestamp(status), timestamps.get(status.name()).asLong());
-		}
-
-		ArrayNode tasks = (ArrayNode) result.get("vertices");
-		int x = 0;
-		for (AccessExecutionJobVertex expectedTask : originalJob.getVerticesTopologically()) {
-			JsonNode task = tasks.get(x);
-
-			Assert.assertEquals(expectedTask.getJobVertexId().toString(), task.get("id").asText());
-			Assert.assertEquals(expectedTask.getName(), task.get("name").asText());
-			Assert.assertEquals(expectedTask.getParallelism(), task.get("parallelism").asInt());
-			Assert.assertEquals(expectedTask.getAggregateState().name(), task.get("status").asText());
-
-			Assert.assertEquals(3, task.get("start-time").asLong());
-			Assert.assertEquals(5, task.get("end-time").asLong());
-			Assert.assertEquals(2, task.get("duration").asLong());
-
-			JsonNode subtasksPerState = task.get("tasks");
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CREATED.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.SCHEDULED.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.DEPLOYING.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.RUNNING.name()).asInt());
-			Assert.assertEquals(1, subtasksPerState.get(ExecutionState.FINISHED.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CANCELING.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CANCELED.name()).asInt());
-			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.FAILED.name()).asInt());
-
-			long expectedNumBytesIn = 0;
-			long expectedNumBytesOut = 0;
-			long expectedNumRecordsIn = 0;
-			long expectedNumRecordsOut = 0;
-
-			for (AccessExecutionVertex vertex : expectedTask.getTaskVertices()) {
-				IOMetrics ioMetrics = vertex.getCurrentExecutionAttempt().getIOMetrics();
-
-				expectedNumBytesIn += ioMetrics.getNumBytesInLocal() + ioMetrics.getNumBytesInRemote();
-				expectedNumBytesOut += ioMetrics.getNumBytesOut();
-				expectedNumRecordsIn += ioMetrics.getNumRecordsIn();
-				expectedNumRecordsOut += ioMetrics.getNumRecordsOut();
-			}
-
-			JsonNode metrics = task.get("metrics");
-
-			Assert.assertEquals(expectedNumBytesIn, metrics.get("read-bytes").asLong());
-			Assert.assertEquals(expectedNumBytesOut, metrics.get("write-bytes").asLong());
-			Assert.assertEquals(expectedNumRecordsIn, metrics.get("read-records").asLong());
-			Assert.assertEquals(expectedNumRecordsOut, metrics.get("write-records").asLong());
-
-			x++;
-		}
-		Assert.assertEquals(1, tasks.size());
-
-		JsonNode statusCounts = result.get("status-counts");
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CREATED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.SCHEDULED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.DEPLOYING.name()).asInt());
-		Assert.assertEquals(1, statusCounts.get(ExecutionState.RUNNING.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.FINISHED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELING.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.FAILED.name()).asInt());
-
-		Assert.assertEquals(ArchivedJobGenerationUtils.MAPPER.readTree(originalJob.getJsonPlan()), result.get("plan"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
deleted file mode 100644
index 6e0f918..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
+++ /dev/null
@@ -1,101 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-import org.apache.flink.util.ExceptionUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobExceptionsHandler.
- */
-public class JobExceptionsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobExceptionsHandler.JobExceptionsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/exceptions", archive.getPath());
-		compareExceptions(originalJob, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobExceptionsHandler handler = new JobExceptionsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/exceptions", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		String json = JobExceptionsHandler.createJobExceptionsJson(originalJob);
-
-		compareExceptions(originalJob, json);
-	}
-
-	private static void compareExceptions(AccessExecutionGraph originalJob, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalJob.getFailureCause().getExceptionAsString(), result.get("root-exception").asText());
-		Assert.assertEquals(originalJob.getFailureCause().getTimestamp(), result.get("timestamp").asLong());
-
-		ArrayNode exceptions = (ArrayNode) result.get("all-exceptions");
-
-		int x = 0;
-		for (AccessExecutionVertex expectedSubtask : originalJob.getAllExecutionVertices()) {
-			if (!expectedSubtask.getFailureCauseAsString().equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
-				JsonNode exception = exceptions.get(x);
-
-				Assert.assertEquals(expectedSubtask.getFailureCauseAsString(), exception.get("exception").asText());
-				Assert.assertEquals(expectedSubtask.getStateTimestamp(ExecutionState.FAILED), exception.get("timestamp").asLong());
-				Assert.assertEquals(expectedSubtask.getTaskNameWithSubtaskIndex(), exception.get("task").asText());
-
-				TaskManagerLocation location = expectedSubtask.getCurrentAssignedResourceLocation();
-				String expectedLocationString = location.getFQDNHostname() + ':' + location.dataPort();
-				Assert.assertEquals(expectedLocationString, exception.get("location").asText());
-			}
-			x++;
-		}
-		Assert.assertEquals(x > JobExceptionsHandler.MAX_NUMBER_EXCEPTION_TO_REPORT, result.get("truncated").asBoolean());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java
deleted file mode 100644
index 94fd5a8..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java
+++ /dev/null
@@ -1,37 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the JobManagerConfigHandler.
- */
-public class JobManagerConfigHandlerTest {
-	@Test
-	public void testGetPaths() {
-		JobManagerConfigHandler handler = new JobManagerConfigHandler(Executors.directExecutor(), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobmanager/config", paths[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java
deleted file mode 100644
index 4a934ec..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java
+++ /dev/null
@@ -1,60 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobPlanHandler.
- */
-public class JobPlanHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobPlanHandler.JobPlanJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/plan", archive.getPath());
-		Assert.assertEquals(originalJob.getJsonPlan(), archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobPlanHandler handler = new JobPlanHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/plan", paths[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java
deleted file mode 100644
index 8c05c83..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java
+++ /dev/null
@@ -1,45 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.List;
-
-/**
- * Tests for the JobStoppingHandler.
- */
-public class JobStoppingHandlerTest extends TestLogger {
-	@Test
-	public void testGetPaths() {
-		JobStoppingHandler handler = new JobStoppingHandler(Executors.directExecutor(), TestingUtils.TIMEOUT());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(2, paths.length);
-		List<String> pathsList = Lists.newArrayList(paths);
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid/stop"));
-		Assert.assertTrue(pathsList.contains("/jobs/:jobid/yarn-stop"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java
deleted file mode 100644
index 5af9aa6..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java
+++ /dev/null
@@ -1,85 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobVertexAccumulatorsHandler.
- */
-public class JobVertexAccumulatorsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobVertexAccumulatorsHandler.JobVertexAccumulatorsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/accumulators", archive.getPath());
-		compareAccumulators(originalTask, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobVertexAccumulatorsHandler handler = new JobVertexAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/accumulators", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		String json = JobVertexAccumulatorsHandler.createVertexAccumulatorsJson(originalTask);
-
-		compareAccumulators(originalTask, json);
-	}
-
-	private static void compareAccumulators(AccessExecutionJobVertex originalTask, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
-
-		ArrayNode accs = (ArrayNode) result.get("user-accumulators");
-		StringifiedAccumulatorResult[] expectedAccs = originalTask.getAggregatedUserAccumulatorsStringified();
-
-		ArchivedJobGenerationUtils.compareStringifiedAccumulators(expectedAccs, accs);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java
deleted file mode 100644
index 0d15e08..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java
+++ /dev/null
@@ -1,211 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.BackPressureStatsTracker;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.OperatorBackPressureStats;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collections;
-
-import scala.Option;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for back pressure handler responses.
- */
-public class JobVertexBackPressureHandlerTest {
-	@Test
-	public void testGetPaths() {
-		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), mock(BackPressureStatsTracker.class), 0);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/backpressure", paths[0]);
-	}
-
-	/** Tests the response when no stats are available. */
-	@Test
-	public void testResponseNoStatsAvailable() throws Exception {
-		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
-		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
-
-		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
-				.thenReturn(Option.<OperatorBackPressureStats>empty());
-
-		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
-				mock(ExecutionGraphHolder.class),
-				Executors.directExecutor(),
-				statsTracker,
-				9999);
-
-		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(response);
-
-		// Single element
-		assertEquals(1, rootNode.size());
-
-		// Status
-		JsonNode status = rootNode.get("status");
-		assertNotNull(status);
-		assertEquals("deprecated", status.textValue());
-
-		verify(statsTracker).triggerStackTraceSample(any(ExecutionJobVertex.class));
-	}
-
-	/** Tests the response when stats are available. */
-	@Test
-	public void testResponseStatsAvailable() throws Exception {
-		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
-		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
-
-		OperatorBackPressureStats stats = new OperatorBackPressureStats(
-				0, System.currentTimeMillis(), new double[] { 0.31, 0.48, 1.0, 0.0 });
-
-		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
-				.thenReturn(Option.apply(stats));
-
-		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
-				mock(ExecutionGraphHolder.class),
-				Executors.directExecutor(),
-				statsTracker,
-				9999);
-
-		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(response);
-
-		// Single element
-		assertEquals(4, rootNode.size());
-
-		// Status
-		JsonNode status = rootNode.get("status");
-		assertNotNull(status);
-		assertEquals("ok", status.textValue());
-
-		// Back pressure level
-		JsonNode backPressureLevel = rootNode.get("backpressure-level");
-		assertNotNull(backPressureLevel);
-		assertEquals("high", backPressureLevel.textValue());
-
-		// End time stamp
-		JsonNode endTimeStamp = rootNode.get("end-timestamp");
-		assertNotNull(endTimeStamp);
-		assertEquals(stats.getEndTimestamp(), endTimeStamp.longValue());
-
-		// Subtasks
-		JsonNode subTasks = rootNode.get("subtasks");
-		assertEquals(stats.getNumberOfSubTasks(), subTasks.size());
-		for (int i = 0; i < subTasks.size(); i++) {
-			JsonNode subTask = subTasks.get(i);
-
-			JsonNode index = subTask.get("subtask");
-			assertEquals(i, index.intValue());
-
-			JsonNode level = subTask.get("backpressure-level");
-			assertEquals(JobVertexBackPressureHandler
-					.getBackPressureLevel(stats.getBackPressureRatio(i)), level.textValue());
-
-			JsonNode ratio = subTask.get("ratio");
-			assertEquals(stats.getBackPressureRatio(i), ratio.doubleValue(), 0.0);
-		}
-
-		// Verify not triggered
-		verify(statsTracker, never()).triggerStackTraceSample(any(ExecutionJobVertex.class));
-	}
-
-	/** Tests that after the refresh interval another sample is triggered. */
-	@Test
-	public void testResponsePassedRefreshInterval() throws Exception {
-		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
-		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
-
-		OperatorBackPressureStats stats = new OperatorBackPressureStats(
-				0, System.currentTimeMillis(), new double[] { 0.31, 0.48, 1.0, 0.0 });
-
-		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
-				.thenReturn(Option.apply(stats));
-
-		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
-				mock(ExecutionGraphHolder.class),
-				Executors.directExecutor(),
-				statsTracker,
-				0); // <----- refresh interval should fire immediately
-
-		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
-
-		ObjectMapper mapper = new ObjectMapper();
-		JsonNode rootNode = mapper.readTree(response);
-
-		// Single element
-		assertEquals(4, rootNode.size());
-
-		// Status
-		JsonNode status = rootNode.get("status");
-		assertNotNull(status);
-		// Interval passed, hence deprecated
-		assertEquals("deprecated", status.textValue());
-
-		// Back pressure level
-		JsonNode backPressureLevel = rootNode.get("backpressure-level");
-		assertNotNull(backPressureLevel);
-		assertEquals("high", backPressureLevel.textValue());
-
-		// End time stamp
-		JsonNode endTimeStamp = rootNode.get("end-timestamp");
-		assertNotNull(endTimeStamp);
-		assertEquals(stats.getEndTimestamp(), endTimeStamp.longValue());
-
-		// Subtasks
-		JsonNode subTasks = rootNode.get("subtasks");
-		assertEquals(stats.getNumberOfSubTasks(), subTasks.size());
-		for (int i = 0; i < subTasks.size(); i++) {
-			JsonNode subTask = subTasks.get(i);
-
-			JsonNode index = subTask.get("subtask");
-			assertEquals(i, index.intValue());
-
-			JsonNode level = subTask.get("backpressure-level");
-			assertEquals(JobVertexBackPressureHandler
-					.getBackPressureLevel(stats.getBackPressureRatio(i)), level.textValue());
-
-			JsonNode ratio = subTask.get("ratio");
-			assertEquals(stats.getBackPressureRatio(i), ratio.doubleValue(), 0.0);
-		}
-
-		// Verify triggered
-		verify(statsTracker).triggerStackTraceSample(any(ExecutionJobVertex.class));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java
deleted file mode 100644
index 1b8d9aa..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java
+++ /dev/null
@@ -1,109 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobVertexDetailsHandler.
- */
-public class JobVertexDetailsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobVertexDetailsHandler.JobVertexDetailsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId(), archive.getPath());
-		compareVertexDetails(originalTask, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobVertexDetailsHandler handler = new JobVertexDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		String json = JobVertexDetailsHandler.createVertexDetailsJson(
-			originalTask, ArchivedJobGenerationUtils.getTestJob().getJobID().toString(), null);
-
-		compareVertexDetails(originalTask, json);
-	}
-
-	private static void compareVertexDetails(AccessExecutionJobVertex originalTask, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
-		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
-		Assert.assertEquals(originalTask.getParallelism(), result.get("parallelism").asInt());
-		Assert.assertTrue(result.get("now").asLong() > 0);
-
-		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
-
-		Assert.assertEquals(originalTask.getTaskVertices().length, subtasks.size());
-		for (int x = 0; x < originalTask.getTaskVertices().length; x++) {
-			AccessExecutionVertex expectedSubtask = originalTask.getTaskVertices()[x];
-			JsonNode subtask = subtasks.get(x);
-
-			Assert.assertEquals(x, subtask.get("subtask").asInt());
-			Assert.assertEquals(expectedSubtask.getExecutionState().name(), subtask.get("status").asText());
-			Assert.assertEquals(expectedSubtask.getCurrentExecutionAttempt().getAttemptNumber(), subtask.get("attempt").asInt());
-
-			TaskManagerLocation location = expectedSubtask.getCurrentAssignedResourceLocation();
-			String expectedLocationString = location.getHostname() + ":" + location.dataPort();
-			Assert.assertEquals(expectedLocationString, subtask.get("host").asText());
-			long start = expectedSubtask.getStateTimestamp(ExecutionState.DEPLOYING);
-			Assert.assertEquals(start, subtask.get("start-time").asLong());
-			long end = expectedSubtask.getStateTimestamp(ExecutionState.FINISHED);
-			Assert.assertEquals(end, subtask.get("end-time").asLong());
-			Assert.assertEquals(end - start, subtask.get("duration").asLong());
-
-			ArchivedJobGenerationUtils.compareIoMetrics(expectedSubtask.getCurrentExecutionAttempt().getIOMetrics(), subtask.get("metrics"));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java
deleted file mode 100644
index badb952..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java
+++ /dev/null
@@ -1,132 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.executiongraph.IOMetrics;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the JobVertexTaskManagersHandler.
- */
-public class JobVertexTaskManagersHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new JobVertexTaskManagersHandler.JobVertexTaskManagersJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecutionVertex originalSubtask = ArchivedJobGenerationUtils.getTestSubtask();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/taskmanagers", archive.getPath());
-		compareVertexTaskManagers(originalTask, originalSubtask, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		JobVertexTaskManagersHandler handler = new JobVertexTaskManagersHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/taskmanagers", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecutionVertex originalSubtask = ArchivedJobGenerationUtils.getTestSubtask();
-		String json = JobVertexTaskManagersHandler.createVertexDetailsByTaskManagerJson(
-			originalTask, ArchivedJobGenerationUtils.getTestJob().getJobID().toString(), null);
-
-		compareVertexTaskManagers(originalTask, originalSubtask, json);
-	}
-
-	private static void compareVertexTaskManagers(AccessExecutionJobVertex originalTask, AccessExecutionVertex originalSubtask, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
-		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
-		Assert.assertTrue(result.get("now").asLong() > 0);
-
-		ArrayNode taskmanagers = (ArrayNode) result.get("taskmanagers");
-
-		JsonNode taskManager = taskmanagers.get(0);
-
-		TaskManagerLocation location = originalSubtask.getCurrentAssignedResourceLocation();
-		String expectedLocationString = location.getHostname() + ':' + location.dataPort();
-		Assert.assertEquals(expectedLocationString, taskManager.get("host").asText());
-		Assert.assertEquals(ExecutionState.FINISHED.name(), taskManager.get("status").asText());
-
-		Assert.assertEquals(3, taskManager.get("start-time").asLong());
-		Assert.assertEquals(5, taskManager.get("end-time").asLong());
-		Assert.assertEquals(2, taskManager.get("duration").asLong());
-
-		JsonNode statusCounts = taskManager.get("status-counts");
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CREATED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.SCHEDULED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.DEPLOYING.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.RUNNING.name()).asInt());
-		Assert.assertEquals(1, statusCounts.get(ExecutionState.FINISHED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELING.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELED.name()).asInt());
-		Assert.assertEquals(0, statusCounts.get(ExecutionState.FAILED.name()).asInt());
-
-		long expectedNumBytesIn = 0;
-		long expectedNumBytesOut = 0;
-		long expectedNumRecordsIn = 0;
-		long expectedNumRecordsOut = 0;
-
-		for (AccessExecutionVertex vertex : originalTask.getTaskVertices()) {
-			IOMetrics ioMetrics = vertex.getCurrentExecutionAttempt().getIOMetrics();
-
-			expectedNumBytesIn += ioMetrics.getNumBytesInLocal() + ioMetrics.getNumBytesInRemote();
-			expectedNumBytesOut += ioMetrics.getNumBytesOut();
-			expectedNumRecordsIn += ioMetrics.getNumRecordsIn();
-			expectedNumRecordsOut += ioMetrics.getNumRecordsOut();
-		}
-
-		JsonNode metrics = taskManager.get("metrics");
-
-		Assert.assertEquals(expectedNumBytesIn, metrics.get("read-bytes").asLong());
-		Assert.assertEquals(expectedNumBytesOut, metrics.get("write-bytes").asLong());
-		Assert.assertEquals(expectedNumRecordsIn, metrics.get("read-records").asLong());
-		Assert.assertEquals(expectedNumRecordsOut, metrics.get("write-records").asLong());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java
deleted file mode 100644
index a80bac9..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java
+++ /dev/null
@@ -1,40 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the SubtaskCurrentAttemptDetailsHandler.
- */
-public class SubtaskCurrentAttemptDetailsHandlerTest {
-	@Test
-	public void testGetPaths() {
-		SubtaskCurrentAttemptDetailsHandler handler = new SubtaskCurrentAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", paths[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
deleted file mode 100644
index 6773fd4..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
+++ /dev/null
@@ -1,91 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the SubtaskExecutionAttemptAccumulatorsHandler.
- */
-public class SubtaskExecutionAttemptAccumulatorsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new SubtaskExecutionAttemptAccumulatorsHandler.SubtaskExecutionAttemptAccumulatorsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals(
-			"/jobs/" + originalJob.getJobID() +
-			"/vertices/" + originalTask.getJobVertexId() +
-			"/subtasks/" + originalAttempt.getParallelSubtaskIndex() +
-			"/attempts/" + originalAttempt.getAttemptNumber() +
-			"/accumulators",
-			archive.getPath());
-		compareAttemptAccumulators(originalAttempt, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		SubtaskExecutionAttemptAccumulatorsHandler handler = new SubtaskExecutionAttemptAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-		String json = SubtaskExecutionAttemptAccumulatorsHandler.createAttemptAccumulatorsJson(originalAttempt);
-
-		compareAttemptAccumulators(originalAttempt, json);
-	}
-
-	private static void compareAttemptAccumulators(AccessExecution originalAttempt, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalAttempt.getParallelSubtaskIndex(), result.get("subtask").asInt());
-		Assert.assertEquals(originalAttempt.getAttemptNumber(), result.get("attempt").asInt());
-		Assert.assertEquals(originalAttempt.getAttemptId().toString(), result.get("id").asText());
-
-		ArchivedJobGenerationUtils.compareStringifiedAccumulators(originalAttempt.getUserAccumulatorsStringified(), (ArrayNode) result.get("user-accumulators"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java
deleted file mode 100644
index 7777d2d..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java
+++ /dev/null
@@ -1,109 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.AccessExecution;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the SubtaskExecutionAttemptDetailsHandler.
- */
-public class SubtaskExecutionAttemptDetailsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new SubtaskExecutionAttemptDetailsHandler.SubtaskExecutionAttemptDetailsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(2, archives.size());
-
-		Iterator<ArchivedJson> iterator = archives.iterator();
-		ArchivedJson archive1 = iterator.next();
-		Assert.assertEquals(
-			"/jobs/" + originalJob.getJobID() +
-				"/vertices/" + originalTask.getJobVertexId() +
-				"/subtasks/" + originalAttempt.getParallelSubtaskIndex(),
-			archive1.getPath());
-		compareAttemptDetails(originalAttempt, archive1.getJson());
-
-		ArchivedJson archive2 = iterator.next();
-		Assert.assertEquals(
-			"/jobs/" + originalJob.getJobID() +
-				"/vertices/" + originalTask.getJobVertexId() +
-				"/subtasks/" + originalAttempt.getParallelSubtaskIndex() +
-				"/attempts/" + originalAttempt.getAttemptNumber(),
-			archive2.getPath());
-		compareAttemptDetails(originalAttempt, archive2.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		SubtaskExecutionAttemptDetailsHandler handler = new SubtaskExecutionAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(),  null);
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
-		String json = SubtaskExecutionAttemptDetailsHandler.createAttemptDetailsJson(
-			originalAttempt, originalJob.getJobID().toString(), originalTask.getJobVertexId().toString(), null);
-
-		compareAttemptDetails(originalAttempt, json);
-	}
-
-	private static void compareAttemptDetails(AccessExecution originalAttempt, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalAttempt.getParallelSubtaskIndex(), result.get("subtask").asInt());
-		Assert.assertEquals(originalAttempt.getState().name(), result.get("status").asText());
-		Assert.assertEquals(originalAttempt.getAttemptNumber(), result.get("attempt").asInt());
-		Assert.assertEquals(originalAttempt.getAssignedResourceLocation().getHostname(), result.get("host").asText());
-		long start = originalAttempt.getStateTimestamp(ExecutionState.DEPLOYING);
-		Assert.assertEquals(start, result.get("start-time").asLong());
-		long end = originalAttempt.getStateTimestamp(ExecutionState.FINISHED);
-		Assert.assertEquals(end, result.get("end-time").asLong());
-		Assert.assertEquals(end - start, result.get("duration").asLong());
-
-		ArchivedJobGenerationUtils.compareIoMetrics(originalAttempt.getIOMetrics(), result.get("metrics"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java
deleted file mode 100644
index 7b400da..0000000
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java
+++ /dev/null
@@ -1,97 +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.webmonitor.handlers;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
-import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
-import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
-import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
-import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
-import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the SubtasksAllAccumulatorsHandler.
- */
-public class SubtasksAllAccumulatorsHandlerTest {
-
-	@Test
-	public void testArchiver() throws Exception {
-		JsonArchivist archivist = new SubtasksAllAccumulatorsHandler.SubtasksAllAccumulatorsJsonArchivist();
-		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-
-		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
-		Assert.assertEquals(1, archives.size());
-
-		ArchivedJson archive = archives.iterator().next();
-		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() +
-			"/subtasks/accumulators", archive.getPath());
-		compareSubtaskAccumulators(originalTask, archive.getJson());
-	}
-
-	@Test
-	public void testGetPaths() {
-		SubtasksAllAccumulatorsHandler handler = new SubtasksAllAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", paths[0]);
-	}
-
-	@Test
-	public void testJsonGeneration() throws Exception {
-		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
-		String json = SubtasksAllAccumulatorsHandler.createSubtasksAccumulatorsJson(originalTask);
-		compareSubtaskAccumulators(originalTask, json);
-	}
-
-	private static void compareSubtaskAccumulators(AccessExecutionJobVertex originalTask, String json) throws IOException {
-		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
-
-		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
-		Assert.assertEquals(originalTask.getParallelism(), result.get("parallelism").asInt());
-
-		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
-
-		Assert.assertEquals(originalTask.getTaskVertices().length, subtasks.size());
-		for (int x = 0; x < originalTask.getTaskVertices().length; x++) {
-			JsonNode subtask = subtasks.get(x);
-			AccessExecutionVertex expectedSubtask = originalTask.getTaskVertices()[x];
-
-			Assert.assertEquals(x, subtask.get("subtask").asInt());
-			Assert.assertEquals(expectedSubtask.getCurrentExecutionAttempt().getAttemptNumber(), subtask.get("attempt").asInt());
-			Assert.assertEquals(expectedSubtask.getCurrentAssignedResourceLocation().getHostname(), subtask.get("host").asText());
-
-			ArchivedJobGenerationUtils.compareStringifiedAccumulators(
-				expectedSubtask.getCurrentExecutionAttempt().getUserAccumulatorsStringified(),
-				(ArrayNode) subtask.get("user-accumulators"));
-		}
-	}
-}


[04/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
new file mode 100644
index 0000000..8f21af0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobCancellationWithSavepointHandlersTest.java
@@ -0,0 +1,333 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the JobCancellationWithSavepointHandler.
+ */
+public class JobCancellationWithSavepointHandlersTest extends TestLogger {
+
+	private static final Executor executor = Executors.directExecutor();
+
+	@Test
+	public void testGetPaths() {
+		JobCancellationWithSavepointHandlers handler = new JobCancellationWithSavepointHandlers(mock(ExecutionGraphHolder.class), executor);
+
+		JobCancellationWithSavepointHandlers.TriggerHandler triggerHandler = handler.getTriggerHandler();
+		String[] triggerPaths = triggerHandler.getPaths();
+		Assert.assertEquals(2, triggerPaths.length);
+		List<String> triggerPathsList = Arrays.asList(triggerPaths);
+		Assert.assertTrue(triggerPathsList.contains("/jobs/:jobid/cancel-with-savepoint"));
+		Assert.assertTrue(triggerPathsList.contains("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory"));
+
+		JobCancellationWithSavepointHandlers.InProgressHandler progressHandler = handler.getInProgressHandler();
+		String[] progressPaths = progressHandler.getPaths();
+		Assert.assertEquals(1, progressPaths.length);
+		Assert.assertEquals("/jobs/:jobid/cancel-with-savepoint/in-progress/:requestId", progressPaths[0]);
+	}
+
+	/**
+	 * Tests that the cancellation ask timeout respects the checkpoint timeout.
+	 * Otherwise, AskTimeoutExceptions are bound to happen for large state.
+	 */
+	@Test
+	public void testAskTimeoutEqualsCheckpointTimeout() throws Exception {
+		long timeout = 128288238L;
+		JobID jobId = new JobID();
+		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
+		ExecutionGraph graph = mock(ExecutionGraph.class);
+		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
+		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
+		when(graph.getCheckpointCoordinator()).thenReturn(coord);
+		when(coord.getCheckpointTimeout()).thenReturn(timeout);
+
+		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
+		JobCancellationWithSavepointHandlers.TriggerHandler handler = handlers.getTriggerHandler();
+
+		Map<String, String> params = new HashMap<>();
+		params.put("jobid", jobId.toString());
+		params.put("targetDirectory", "placeholder");
+
+		JobManagerGateway jobManager = mock(JobManagerGateway.class);
+		when(jobManager.cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class))).thenReturn(CompletableFuture.completedFuture("foobar"));
+
+		handler.handleRequest(params, Collections.emptyMap(), jobManager);
+
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class));
+	}
+
+	/**
+	 * Tests that the savepoint directory configuration is respected.
+	 */
+	@Test
+	public void testSavepointDirectoryConfiguration() throws Exception {
+		long timeout = 128288238L;
+		JobID jobId = new JobID();
+		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
+		ExecutionGraph graph = mock(ExecutionGraph.class);
+		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
+		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
+		when(graph.getCheckpointCoordinator()).thenReturn(coord);
+		when(coord.getCheckpointTimeout()).thenReturn(timeout);
+
+		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor, "the-default-directory");
+		JobCancellationWithSavepointHandlers.TriggerHandler handler = handlers.getTriggerHandler();
+
+		Map<String, String> params = new HashMap<>();
+		params.put("jobid", jobId.toString());
+
+		JobManagerGateway jobManager = mock(JobManagerGateway.class);
+		when(jobManager.cancelJobWithSavepoint(eq(jobId), anyString(), any(Time.class))).thenReturn(CompletableFuture.completedFuture("foobar"));
+
+		// 1. Use targetDirectory path param
+		params.put("targetDirectory", "custom-directory");
+		handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
+
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
+
+		// 2. Use default
+		params.remove("targetDirectory");
+
+		handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
+
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("the-default-directory"), any(Time.class));
+
+		// 3. Throw Exception
+		handlers = new JobCancellationWithSavepointHandlers(holder, executor, null);
+		handler = handlers.getTriggerHandler();
+
+		try {
+			handler.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+			fail("Did not throw expected test Exception");
+		} catch (Exception e) {
+			IllegalStateException cause = (IllegalStateException) e.getCause();
+			assertEquals(true, cause.getMessage().contains(CoreOptions.SAVEPOINT_DIRECTORY.key()));
+		}
+	}
+
+	/**
+	 * Tests triggering a new request and monitoring it.
+	 */
+	@Test
+	public void testTriggerNewRequest() throws Exception {
+		JobID jobId = new JobID();
+		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
+		ExecutionGraph graph = mock(ExecutionGraph.class);
+		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
+		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
+		when(graph.getCheckpointCoordinator()).thenReturn(coord);
+
+		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
+		JobCancellationWithSavepointHandlers.TriggerHandler trigger = handlers.getTriggerHandler();
+		JobCancellationWithSavepointHandlers.InProgressHandler progress = handlers.getInProgressHandler();
+
+		Map<String, String> params = new HashMap<>();
+		params.put("jobid", jobId.toString());
+		params.put("targetDirectory", "custom-directory");
+
+		JobManagerGateway jobManager = mock(JobManagerGateway.class);
+
+		// Successful
+		CompletableFuture<String> successfulCancelWithSavepoint = new CompletableFuture<>();
+		when(jobManager.cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class))).thenReturn(successfulCancelWithSavepoint);
+
+		// Trigger
+		FullHttpResponse response = trigger.handleRequest(params, Collections.emptyMap(), jobManager).get();
+
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
+
+		String location = String.format("/jobs/%s/cancel-with-savepoint/in-progress/1", jobId);
+
+		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+		assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
+
+		String json = response.content().toString(Charset.forName("UTF-8"));
+		JsonNode root = new ObjectMapper().readTree(json);
+
+		assertEquals("accepted", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+		assertEquals(location, root.get("location").asText());
+
+		// Trigger again
+		response = trigger.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+		assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
+
+		json = response.content().toString(Charset.forName("UTF-8"));
+		root = new ObjectMapper().readTree(json);
+
+		assertEquals("accepted", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+		assertEquals(location, root.get("location").asText());
+
+		// Only single actual request
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
+
+		// Query progress
+		params.put("requestId", "1");
+
+		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+		assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+
+		json = response.content().toString(Charset.forName("UTF-8"));
+		root = new ObjectMapper().readTree(json);
+
+		assertEquals("in-progress", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+
+		// Complete
+		successfulCancelWithSavepoint.complete("_path-savepoint_");
+
+		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+
+		assertEquals(HttpResponseStatus.CREATED, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+
+		json = response.content().toString(Charset.forName("UTF-8"));
+
+		root = new ObjectMapper().readTree(json);
+
+		assertEquals("success", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+		assertEquals("_path-savepoint_", root.get("savepoint-path").asText());
+
+		// Query again, keep recent history
+
+		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+
+		assertEquals(HttpResponseStatus.CREATED, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+
+		json = response.content().toString(Charset.forName("UTF-8"));
+
+		root = new ObjectMapper().readTree(json);
+
+		assertEquals("success", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+		assertEquals("_path-savepoint_", root.get("savepoint-path").asText());
+
+		// Query for unknown request
+		params.put("requestId", "9929");
+
+		response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+		assertEquals(HttpResponseStatus.BAD_REQUEST, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+
+		json = response.content().toString(Charset.forName("UTF-8"));
+
+		root = new ObjectMapper().readTree(json);
+
+		assertEquals("failed", root.get("status").asText());
+		assertEquals("9929", root.get("request-id").asText());
+		assertEquals("Unknown job/request ID", root.get("cause").asText());
+	}
+
+	/**
+	 * Tests response when a request fails.
+	 */
+	@Test
+	public void testFailedCancellation() throws Exception {
+		JobID jobId = new JobID();
+		ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class);
+		ExecutionGraph graph = mock(ExecutionGraph.class);
+		CheckpointCoordinator coord = mock(CheckpointCoordinator.class);
+		when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph)));
+		when(graph.getCheckpointCoordinator()).thenReturn(coord);
+
+		JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor);
+		JobCancellationWithSavepointHandlers.TriggerHandler trigger = handlers.getTriggerHandler();
+		JobCancellationWithSavepointHandlers.InProgressHandler progress = handlers.getInProgressHandler();
+
+		Map<String, String> params = new HashMap<>();
+		params.put("jobid", jobId.toString());
+		params.put("targetDirectory", "custom-directory");
+
+		JobManagerGateway jobManager = mock(JobManagerGateway.class);
+
+		// Successful
+		CompletableFuture<String> unsuccessfulCancelWithSavepoint = FutureUtils.completedExceptionally(new Exception("Test Exception"));
+		when(jobManager.cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class))).thenReturn(unsuccessfulCancelWithSavepoint);
+
+		// Trigger
+		trigger.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
+		verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class));
+
+		// Query progress
+		params.put("requestId", "1");
+
+		FullHttpResponse response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager).get();
+		assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR, response.getStatus());
+		assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+		assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
+
+		String json = response.content().toString(Charset.forName("UTF-8"));
+		JsonNode root = new ObjectMapper().readTree(json);
+
+		assertEquals("failed", root.get("status").asText());
+		assertEquals("1", root.get("request-id").asText());
+		assertEquals("Test Exception", root.get("cause").asText());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandlerTest.java
new file mode 100644
index 0000000..567df8c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobConfigHandlerTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.ArchivedExecutionConfig;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobConfigHandler.
+ */
+public class JobConfigHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobConfigHandler.JobConfigJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/config", archive.getPath());
+		compareJobConfig(originalJob, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobConfigHandler handler = new JobConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/config", paths[0]);
+	}
+
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		String answer = JobConfigHandler.createJobConfigJson(originalJob);
+		compareJobConfig(originalJob, answer);
+	}
+
+	private static void compareJobConfig(AccessExecutionGraph originalJob, String answer) throws IOException {
+		JsonNode job = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
+
+		Assert.assertEquals(originalJob.getJobID().toString(), job.get("jid").asText());
+		Assert.assertEquals(originalJob.getJobName(), job.get("name").asText());
+
+		ArchivedExecutionConfig originalConfig = originalJob.getArchivedExecutionConfig();
+		JsonNode config = job.get("execution-config");
+
+		Assert.assertEquals(originalConfig.getExecutionMode(), config.get("execution-mode").asText());
+		Assert.assertEquals(originalConfig.getRestartStrategyDescription(), config.get("restart-strategy").asText());
+		Assert.assertEquals(originalConfig.getParallelism(), config.get("job-parallelism").asInt());
+		Assert.assertEquals(originalConfig.getObjectReuseEnabled(), config.get("object-reuse-mode").asBoolean());
+
+		Map<String, String> originalUserConfig = originalConfig.getGlobalJobParameters();
+		JsonNode userConfig = config.get("user-config");
+
+		for (Map.Entry<String, String> originalEntry : originalUserConfig.entrySet()) {
+			Assert.assertEquals(originalEntry.getValue(), userConfig.get(originalEntry.getKey()).asText());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandlerTest.java
new file mode 100644
index 0000000..afd743e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobDetailsHandlerTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobDetailsHandler.
+ */
+public class JobDetailsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobDetailsHandler.JobDetailsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(2, archives.size());
+
+		Iterator<ArchivedJson> iterator = archives.iterator();
+		ArchivedJson archive1 = iterator.next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID(), archive1.getPath());
+		compareJobDetails(originalJob, archive1.getJson());
+
+		ArchivedJson archive2 = iterator.next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices", archive2.getPath());
+		compareJobDetails(originalJob, archive2.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobDetailsHandler handler = new JobDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(2, paths.length);
+		List<String> pathsList = Lists.newArrayList(paths);
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid"));
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid/vertices"));
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		String json = JobDetailsHandler.createJobDetailsJson(originalJob, null);
+
+		compareJobDetails(originalJob, json);
+	}
+
+	private static void compareJobDetails(AccessExecutionGraph originalJob, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalJob.getJobID().toString(), result.get("jid").asText());
+		Assert.assertEquals(originalJob.getJobName(), result.get("name").asText());
+		Assert.assertEquals(originalJob.isStoppable(), result.get("isStoppable").asBoolean());
+		Assert.assertEquals(originalJob.getState().name(), result.get("state").asText());
+
+		Assert.assertEquals(originalJob.getStatusTimestamp(JobStatus.CREATED), result.get("start-time").asLong());
+		Assert.assertEquals(originalJob.getStatusTimestamp(originalJob.getState()), result.get("end-time").asLong());
+		Assert.assertEquals(
+			originalJob.getStatusTimestamp(originalJob.getState()) - originalJob.getStatusTimestamp(JobStatus.CREATED),
+			result.get("duration").asLong()
+		);
+
+		JsonNode timestamps = result.get("timestamps");
+		for (JobStatus status : JobStatus.values()) {
+			Assert.assertEquals(originalJob.getStatusTimestamp(status), timestamps.get(status.name()).asLong());
+		}
+
+		ArrayNode tasks = (ArrayNode) result.get("vertices");
+		int x = 0;
+		for (AccessExecutionJobVertex expectedTask : originalJob.getVerticesTopologically()) {
+			JsonNode task = tasks.get(x);
+
+			Assert.assertEquals(expectedTask.getJobVertexId().toString(), task.get("id").asText());
+			Assert.assertEquals(expectedTask.getName(), task.get("name").asText());
+			Assert.assertEquals(expectedTask.getParallelism(), task.get("parallelism").asInt());
+			Assert.assertEquals(expectedTask.getAggregateState().name(), task.get("status").asText());
+
+			Assert.assertEquals(3, task.get("start-time").asLong());
+			Assert.assertEquals(5, task.get("end-time").asLong());
+			Assert.assertEquals(2, task.get("duration").asLong());
+
+			JsonNode subtasksPerState = task.get("tasks");
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CREATED.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.SCHEDULED.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.DEPLOYING.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.RUNNING.name()).asInt());
+			Assert.assertEquals(1, subtasksPerState.get(ExecutionState.FINISHED.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CANCELING.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.CANCELED.name()).asInt());
+			Assert.assertEquals(0, subtasksPerState.get(ExecutionState.FAILED.name()).asInt());
+
+			long expectedNumBytesIn = 0;
+			long expectedNumBytesOut = 0;
+			long expectedNumRecordsIn = 0;
+			long expectedNumRecordsOut = 0;
+
+			for (AccessExecutionVertex vertex : expectedTask.getTaskVertices()) {
+				IOMetrics ioMetrics = vertex.getCurrentExecutionAttempt().getIOMetrics();
+
+				expectedNumBytesIn += ioMetrics.getNumBytesInLocal() + ioMetrics.getNumBytesInRemote();
+				expectedNumBytesOut += ioMetrics.getNumBytesOut();
+				expectedNumRecordsIn += ioMetrics.getNumRecordsIn();
+				expectedNumRecordsOut += ioMetrics.getNumRecordsOut();
+			}
+
+			JsonNode metrics = task.get("metrics");
+
+			Assert.assertEquals(expectedNumBytesIn, metrics.get("read-bytes").asLong());
+			Assert.assertEquals(expectedNumBytesOut, metrics.get("write-bytes").asLong());
+			Assert.assertEquals(expectedNumRecordsIn, metrics.get("read-records").asLong());
+			Assert.assertEquals(expectedNumRecordsOut, metrics.get("write-records").asLong());
+
+			x++;
+		}
+		Assert.assertEquals(1, tasks.size());
+
+		JsonNode statusCounts = result.get("status-counts");
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CREATED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.SCHEDULED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.DEPLOYING.name()).asInt());
+		Assert.assertEquals(1, statusCounts.get(ExecutionState.RUNNING.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.FINISHED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELING.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.FAILED.name()).asInt());
+
+		Assert.assertEquals(ArchivedJobGenerationUtils.MAPPER.readTree(originalJob.getJsonPlan()), result.get("plan"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandlerTest.java
new file mode 100644
index 0000000..6a20696
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandlerTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.ExceptionUtils;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobExceptionsHandler.
+ */
+public class JobExceptionsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobExceptionsHandler.JobExceptionsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/exceptions", archive.getPath());
+		compareExceptions(originalJob, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobExceptionsHandler handler = new JobExceptionsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/exceptions", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		String json = JobExceptionsHandler.createJobExceptionsJson(originalJob);
+
+		compareExceptions(originalJob, json);
+	}
+
+	private static void compareExceptions(AccessExecutionGraph originalJob, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalJob.getFailureCause().getExceptionAsString(), result.get("root-exception").asText());
+		Assert.assertEquals(originalJob.getFailureCause().getTimestamp(), result.get("timestamp").asLong());
+
+		ArrayNode exceptions = (ArrayNode) result.get("all-exceptions");
+
+		int x = 0;
+		for (AccessExecutionVertex expectedSubtask : originalJob.getAllExecutionVertices()) {
+			if (!expectedSubtask.getFailureCauseAsString().equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
+				JsonNode exception = exceptions.get(x);
+
+				Assert.assertEquals(expectedSubtask.getFailureCauseAsString(), exception.get("exception").asText());
+				Assert.assertEquals(expectedSubtask.getStateTimestamp(ExecutionState.FAILED), exception.get("timestamp").asLong());
+				Assert.assertEquals(expectedSubtask.getTaskNameWithSubtaskIndex(), exception.get("task").asText());
+
+				TaskManagerLocation location = expectedSubtask.getCurrentAssignedResourceLocation();
+				String expectedLocationString = location.getFQDNHostname() + ':' + location.dataPort();
+				Assert.assertEquals(expectedLocationString, exception.get("location").asText());
+			}
+			x++;
+		}
+		Assert.assertEquals(x > JobExceptionsHandler.MAX_NUMBER_EXCEPTION_TO_REPORT, result.get("truncated").asBoolean());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
new file mode 100644
index 0000000..03ddb73
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the JobManagerConfigHandler.
+ */
+public class JobManagerConfigHandlerTest {
+	@Test
+	public void testGetPaths() {
+		JobManagerConfigHandler handler = new JobManagerConfigHandler(Executors.directExecutor(), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobmanager/config", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandlerTest.java
new file mode 100644
index 0000000..6d3b213
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandlerTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobPlanHandler.
+ */
+public class JobPlanHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobPlanHandler.JobPlanJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/plan", archive.getPath());
+		Assert.assertEquals(originalJob.getJsonPlan(), archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobPlanHandler handler = new JobPlanHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/plan", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandlerTest.java
new file mode 100644
index 0000000..2c39fcf
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandlerTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Tests for the JobStoppingHandler.
+ */
+public class JobStoppingHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		JobStoppingHandler handler = new JobStoppingHandler(Executors.directExecutor(), TestingUtils.TIMEOUT());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(2, paths.length);
+		List<String> pathsList = Lists.newArrayList(paths);
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid/stop"));
+		Assert.assertTrue(pathsList.contains("/jobs/:jobid/yarn-stop"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandlerTest.java
new file mode 100644
index 0000000..feffe60
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandlerTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobVertexAccumulatorsHandler.
+ */
+public class JobVertexAccumulatorsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobVertexAccumulatorsHandler.JobVertexAccumulatorsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/accumulators", archive.getPath());
+		compareAccumulators(originalTask, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobVertexAccumulatorsHandler handler = new JobVertexAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/accumulators", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		String json = JobVertexAccumulatorsHandler.createVertexAccumulatorsJson(originalTask);
+
+		compareAccumulators(originalTask, json);
+	}
+
+	private static void compareAccumulators(AccessExecutionJobVertex originalTask, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
+
+		ArrayNode accs = (ArrayNode) result.get("user-accumulators");
+		StringifiedAccumulatorResult[] expectedAccs = originalTask.getAggregatedUserAccumulatorsStringified();
+
+		ArchivedJobGenerationUtils.compareStringifiedAccumulators(expectedAccs, accs);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandlerTest.java
new file mode 100644
index 0000000..bd6817f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandlerTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for back pressure handler responses.
+ */
+public class JobVertexBackPressureHandlerTest {
+	@Test
+	public void testGetPaths() {
+		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), mock(BackPressureStatsTracker.class), 0);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/backpressure", paths[0]);
+	}
+
+	/** Tests the response when no stats are available. */
+	@Test
+	public void testResponseNoStatsAvailable() throws Exception {
+		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
+		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
+
+		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
+				.thenReturn(Optional.empty());
+
+		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
+				mock(ExecutionGraphHolder.class),
+				Executors.directExecutor(),
+				statsTracker,
+				9999);
+
+		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(response);
+
+		// Single element
+		assertEquals(1, rootNode.size());
+
+		// Status
+		JsonNode status = rootNode.get("status");
+		assertNotNull(status);
+		assertEquals("deprecated", status.textValue());
+
+		verify(statsTracker).triggerStackTraceSample(any(ExecutionJobVertex.class));
+	}
+
+	/** Tests the response when stats are available. */
+	@Test
+	public void testResponseStatsAvailable() throws Exception {
+		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
+		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
+
+		OperatorBackPressureStats stats = new OperatorBackPressureStats(
+				0, System.currentTimeMillis(), new double[] { 0.31, 0.48, 1.0, 0.0 });
+
+		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
+				.thenReturn(Optional.of(stats));
+
+		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
+				mock(ExecutionGraphHolder.class),
+				Executors.directExecutor(),
+				statsTracker,
+				9999);
+
+		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(response);
+
+		// Single element
+		assertEquals(4, rootNode.size());
+
+		// Status
+		JsonNode status = rootNode.get("status");
+		assertNotNull(status);
+		assertEquals("ok", status.textValue());
+
+		// Back pressure level
+		JsonNode backPressureLevel = rootNode.get("backpressure-level");
+		assertNotNull(backPressureLevel);
+		assertEquals("high", backPressureLevel.textValue());
+
+		// End time stamp
+		JsonNode endTimeStamp = rootNode.get("end-timestamp");
+		assertNotNull(endTimeStamp);
+		assertEquals(stats.getEndTimestamp(), endTimeStamp.longValue());
+
+		// Subtasks
+		JsonNode subTasks = rootNode.get("subtasks");
+		assertEquals(stats.getNumberOfSubTasks(), subTasks.size());
+		for (int i = 0; i < subTasks.size(); i++) {
+			JsonNode subTask = subTasks.get(i);
+
+			JsonNode index = subTask.get("subtask");
+			assertEquals(i, index.intValue());
+
+			JsonNode level = subTask.get("backpressure-level");
+			assertEquals(JobVertexBackPressureHandler
+					.getBackPressureLevel(stats.getBackPressureRatio(i)), level.textValue());
+
+			JsonNode ratio = subTask.get("ratio");
+			assertEquals(stats.getBackPressureRatio(i), ratio.doubleValue(), 0.0);
+		}
+
+		// Verify not triggered
+		verify(statsTracker, never()).triggerStackTraceSample(any(ExecutionJobVertex.class));
+	}
+
+	/** Tests that after the refresh interval another sample is triggered. */
+	@Test
+	public void testResponsePassedRefreshInterval() throws Exception {
+		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
+		BackPressureStatsTracker statsTracker = mock(BackPressureStatsTracker.class);
+
+		OperatorBackPressureStats stats = new OperatorBackPressureStats(
+				0, System.currentTimeMillis(), new double[] { 0.31, 0.48, 1.0, 0.0 });
+
+		when(statsTracker.getOperatorBackPressureStats(any(ExecutionJobVertex.class)))
+				.thenReturn(Optional.of(stats));
+
+		JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(
+				mock(ExecutionGraphHolder.class),
+				Executors.directExecutor(),
+				statsTracker,
+				0); // <----- refresh interval should fire immediately
+
+		String response = handler.handleRequest(jobVertex, Collections.<String, String>emptyMap()).get();
+
+		ObjectMapper mapper = new ObjectMapper();
+		JsonNode rootNode = mapper.readTree(response);
+
+		// Single element
+		assertEquals(4, rootNode.size());
+
+		// Status
+		JsonNode status = rootNode.get("status");
+		assertNotNull(status);
+		// Interval passed, hence deprecated
+		assertEquals("deprecated", status.textValue());
+
+		// Back pressure level
+		JsonNode backPressureLevel = rootNode.get("backpressure-level");
+		assertNotNull(backPressureLevel);
+		assertEquals("high", backPressureLevel.textValue());
+
+		// End time stamp
+		JsonNode endTimeStamp = rootNode.get("end-timestamp");
+		assertNotNull(endTimeStamp);
+		assertEquals(stats.getEndTimestamp(), endTimeStamp.longValue());
+
+		// Subtasks
+		JsonNode subTasks = rootNode.get("subtasks");
+		assertEquals(stats.getNumberOfSubTasks(), subTasks.size());
+		for (int i = 0; i < subTasks.size(); i++) {
+			JsonNode subTask = subTasks.get(i);
+
+			JsonNode index = subTask.get("subtask");
+			assertEquals(i, index.intValue());
+
+			JsonNode level = subTask.get("backpressure-level");
+			assertEquals(JobVertexBackPressureHandler
+					.getBackPressureLevel(stats.getBackPressureRatio(i)), level.textValue());
+
+			JsonNode ratio = subTask.get("ratio");
+			assertEquals(stats.getBackPressureRatio(i), ratio.doubleValue(), 0.0);
+		}
+
+		// Verify triggered
+		verify(statsTracker).triggerStackTraceSample(any(ExecutionJobVertex.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandlerTest.java
new file mode 100644
index 0000000..5af1d53
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandlerTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobVertexDetailsHandler.
+ */
+public class JobVertexDetailsHandlerTest {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobVertexDetailsHandler.JobVertexDetailsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId(), archive.getPath());
+		compareVertexDetails(originalTask, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobVertexDetailsHandler handler = new JobVertexDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		String json = JobVertexDetailsHandler.createVertexDetailsJson(
+			originalTask, ArchivedJobGenerationUtils.getTestJob().getJobID().toString(), null);
+
+		compareVertexDetails(originalTask, json);
+	}
+
+	private static void compareVertexDetails(AccessExecutionJobVertex originalTask, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
+		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
+		Assert.assertEquals(originalTask.getParallelism(), result.get("parallelism").asInt());
+		Assert.assertTrue(result.get("now").asLong() > 0);
+
+		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
+
+		Assert.assertEquals(originalTask.getTaskVertices().length, subtasks.size());
+		for (int x = 0; x < originalTask.getTaskVertices().length; x++) {
+			AccessExecutionVertex expectedSubtask = originalTask.getTaskVertices()[x];
+			JsonNode subtask = subtasks.get(x);
+
+			Assert.assertEquals(x, subtask.get("subtask").asInt());
+			Assert.assertEquals(expectedSubtask.getExecutionState().name(), subtask.get("status").asText());
+			Assert.assertEquals(expectedSubtask.getCurrentExecutionAttempt().getAttemptNumber(), subtask.get("attempt").asInt());
+
+			TaskManagerLocation location = expectedSubtask.getCurrentAssignedResourceLocation();
+			String expectedLocationString = location.getHostname() + ":" + location.dataPort();
+			Assert.assertEquals(expectedLocationString, subtask.get("host").asText());
+			long start = expectedSubtask.getStateTimestamp(ExecutionState.DEPLOYING);
+			Assert.assertEquals(start, subtask.get("start-time").asLong());
+			long end = expectedSubtask.getStateTimestamp(ExecutionState.FINISHED);
+			Assert.assertEquals(end, subtask.get("end-time").asLong());
+			Assert.assertEquals(end - start, subtask.get("duration").asLong());
+
+			ArchivedJobGenerationUtils.compareIoMetrics(expectedSubtask.getCurrentExecutionAttempt().getIOMetrics(), subtask.get("metrics"));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandlerTest.java
new file mode 100644
index 0000000..2a027fd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandlerTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.TestLogger;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the JobVertexTaskManagersHandler.
+ */
+public class JobVertexTaskManagersHandlerTest extends TestLogger {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new JobVertexTaskManagersHandler.JobVertexTaskManagersJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecutionVertex originalSubtask = ArchivedJobGenerationUtils.getTestSubtask();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() + "/taskmanagers", archive.getPath());
+		compareVertexTaskManagers(originalTask, originalSubtask, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		JobVertexTaskManagersHandler handler = new JobVertexTaskManagersHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/taskmanagers", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecutionVertex originalSubtask = ArchivedJobGenerationUtils.getTestSubtask();
+		String json = JobVertexTaskManagersHandler.createVertexDetailsByTaskManagerJson(
+			originalTask, ArchivedJobGenerationUtils.getTestJob().getJobID().toString(), null);
+
+		compareVertexTaskManagers(originalTask, originalSubtask, json);
+	}
+
+	private static void compareVertexTaskManagers(AccessExecutionJobVertex originalTask, AccessExecutionVertex originalSubtask, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
+		Assert.assertEquals(originalTask.getName(), result.get("name").asText());
+		Assert.assertTrue(result.get("now").asLong() > 0);
+
+		ArrayNode taskmanagers = (ArrayNode) result.get("taskmanagers");
+
+		JsonNode taskManager = taskmanagers.get(0);
+
+		TaskManagerLocation location = originalSubtask.getCurrentAssignedResourceLocation();
+		String expectedLocationString = location.getHostname() + ':' + location.dataPort();
+		Assert.assertEquals(expectedLocationString, taskManager.get("host").asText());
+		Assert.assertEquals(ExecutionState.FINISHED.name(), taskManager.get("status").asText());
+
+		Assert.assertEquals(3, taskManager.get("start-time").asLong());
+		Assert.assertEquals(5, taskManager.get("end-time").asLong());
+		Assert.assertEquals(2, taskManager.get("duration").asLong());
+
+		JsonNode statusCounts = taskManager.get("status-counts");
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CREATED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.SCHEDULED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.DEPLOYING.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.RUNNING.name()).asInt());
+		Assert.assertEquals(1, statusCounts.get(ExecutionState.FINISHED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELING.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELED.name()).asInt());
+		Assert.assertEquals(0, statusCounts.get(ExecutionState.FAILED.name()).asInt());
+
+		long expectedNumBytesIn = 0;
+		long expectedNumBytesOut = 0;
+		long expectedNumRecordsIn = 0;
+		long expectedNumRecordsOut = 0;
+
+		for (AccessExecutionVertex vertex : originalTask.getTaskVertices()) {
+			IOMetrics ioMetrics = vertex.getCurrentExecutionAttempt().getIOMetrics();
+
+			expectedNumBytesIn += ioMetrics.getNumBytesInLocal() + ioMetrics.getNumBytesInRemote();
+			expectedNumBytesOut += ioMetrics.getNumBytesOut();
+			expectedNumRecordsIn += ioMetrics.getNumRecordsIn();
+			expectedNumRecordsOut += ioMetrics.getNumRecordsOut();
+		}
+
+		JsonNode metrics = taskManager.get("metrics");
+
+		Assert.assertEquals(expectedNumBytesIn, metrics.get("read-bytes").asLong());
+		Assert.assertEquals(expectedNumBytesOut, metrics.get("write-bytes").asLong());
+		Assert.assertEquals(expectedNumRecordsIn, metrics.get("read-records").asLong());
+		Assert.assertEquals(expectedNumRecordsOut, metrics.get("write-records").asLong());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandlerTest.java
new file mode 100644
index 0000000..9e0d549
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandlerTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SubtaskCurrentAttemptDetailsHandler.
+ */
+public class SubtaskCurrentAttemptDetailsHandlerTest extends TestLogger {
+	@Test
+	public void testGetPaths() {
+		SubtaskCurrentAttemptDetailsHandler handler = new SubtaskCurrentAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
new file mode 100644
index 0000000..49e54c0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandlerTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.TestLogger;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SubtaskExecutionAttemptAccumulatorsHandler.
+ */
+public class SubtaskExecutionAttemptAccumulatorsHandlerTest extends TestLogger {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new SubtaskExecutionAttemptAccumulatorsHandler.SubtaskExecutionAttemptAccumulatorsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals(
+			"/jobs/" + originalJob.getJobID() +
+			"/vertices/" + originalTask.getJobVertexId() +
+			"/subtasks/" + originalAttempt.getParallelSubtaskIndex() +
+			"/attempts/" + originalAttempt.getAttemptNumber() +
+			"/accumulators",
+			archive.getPath());
+		compareAttemptAccumulators(originalAttempt, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		SubtaskExecutionAttemptAccumulatorsHandler handler = new SubtaskExecutionAttemptAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+		String json = SubtaskExecutionAttemptAccumulatorsHandler.createAttemptAccumulatorsJson(originalAttempt);
+
+		compareAttemptAccumulators(originalAttempt, json);
+	}
+
+	private static void compareAttemptAccumulators(AccessExecution originalAttempt, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalAttempt.getParallelSubtaskIndex(), result.get("subtask").asInt());
+		Assert.assertEquals(originalAttempt.getAttemptNumber(), result.get("attempt").asInt());
+		Assert.assertEquals(originalAttempt.getAttemptId().toString(), result.get("id").asText());
+
+		ArchivedJobGenerationUtils.compareStringifiedAccumulators(originalAttempt.getUserAccumulatorsStringified(), (ArrayNode) result.get("user-accumulators"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandlerTest.java
new file mode 100644
index 0000000..e1fe8b5
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandlerTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.TestLogger;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SubtaskExecutionAttemptDetailsHandler.
+ */
+public class SubtaskExecutionAttemptDetailsHandlerTest extends TestLogger {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new SubtaskExecutionAttemptDetailsHandler.SubtaskExecutionAttemptDetailsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(2, archives.size());
+
+		Iterator<ArchivedJson> iterator = archives.iterator();
+		ArchivedJson archive1 = iterator.next();
+		Assert.assertEquals(
+			"/jobs/" + originalJob.getJobID() +
+				"/vertices/" + originalTask.getJobVertexId() +
+				"/subtasks/" + originalAttempt.getParallelSubtaskIndex(),
+			archive1.getPath());
+		compareAttemptDetails(originalAttempt, archive1.getJson());
+
+		ArchivedJson archive2 = iterator.next();
+		Assert.assertEquals(
+			"/jobs/" + originalJob.getJobID() +
+				"/vertices/" + originalTask.getJobVertexId() +
+				"/subtasks/" + originalAttempt.getParallelSubtaskIndex() +
+				"/attempts/" + originalAttempt.getAttemptNumber(),
+			archive2.getPath());
+		compareAttemptDetails(originalAttempt, archive2.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		SubtaskExecutionAttemptDetailsHandler handler = new SubtaskExecutionAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(),  null);
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		AccessExecution originalAttempt = ArchivedJobGenerationUtils.getTestAttempt();
+		String json = SubtaskExecutionAttemptDetailsHandler.createAttemptDetailsJson(
+			originalAttempt, originalJob.getJobID().toString(), originalTask.getJobVertexId().toString(), null);
+
+		compareAttemptDetails(originalAttempt, json);
+	}
+
+	private static void compareAttemptDetails(AccessExecution originalAttempt, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalAttempt.getParallelSubtaskIndex(), result.get("subtask").asInt());
+		Assert.assertEquals(originalAttempt.getState().name(), result.get("status").asText());
+		Assert.assertEquals(originalAttempt.getAttemptNumber(), result.get("attempt").asInt());
+		Assert.assertEquals(originalAttempt.getAssignedResourceLocation().getHostname(), result.get("host").asText());
+		long start = originalAttempt.getStateTimestamp(ExecutionState.DEPLOYING);
+		Assert.assertEquals(start, result.get("start-time").asLong());
+		long end = originalAttempt.getStateTimestamp(ExecutionState.FINISHED);
+		Assert.assertEquals(end, result.get("end-time").asLong());
+		Assert.assertEquals(end - start, result.get("duration").asLong());
+
+		ArchivedJobGenerationUtils.compareIoMetrics(originalAttempt.getIOMetrics(), result.get("metrics"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandlerTest.java
new file mode 100644
index 0000000..1478f00
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandlerTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedJobGenerationUtils;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.TestLogger;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SubtasksAllAccumulatorsHandler.
+ */
+public class SubtasksAllAccumulatorsHandlerTest extends TestLogger {
+
+	@Test
+	public void testArchiver() throws Exception {
+		JsonArchivist archivist = new SubtasksAllAccumulatorsHandler.SubtasksAllAccumulatorsJsonArchivist();
+		AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+
+		Collection<ArchivedJson> archives = archivist.archiveJsonWithPath(originalJob);
+		Assert.assertEquals(1, archives.size());
+
+		ArchivedJson archive = archives.iterator().next();
+		Assert.assertEquals("/jobs/" + originalJob.getJobID() + "/vertices/" + originalTask.getJobVertexId() +
+			"/subtasks/accumulators", archive.getPath());
+		compareSubtaskAccumulators(originalTask, archive.getJson());
+	}
+
+	@Test
+	public void testGetPaths() {
+		SubtasksAllAccumulatorsHandler handler = new SubtasksAllAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", paths[0]);
+	}
+
+	@Test
+	public void testJsonGeneration() throws Exception {
+		AccessExecutionJobVertex originalTask = ArchivedJobGenerationUtils.getTestTask();
+		String json = SubtasksAllAccumulatorsHandler.createSubtasksAccumulatorsJson(originalTask);
+		compareSubtaskAccumulators(originalTask, json);
+	}
+
+	private static void compareSubtaskAccumulators(AccessExecutionJobVertex originalTask, String json) throws IOException {
+		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
+
+		Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
+		Assert.assertEquals(originalTask.getParallelism(), result.get("parallelism").asInt());
+
+		ArrayNode subtasks = (ArrayNode) result.get("subtasks");
+
+		Assert.assertEquals(originalTask.getTaskVertices().length, subtasks.size());
+		for (int x = 0; x < originalTask.getTaskVertices().length; x++) {
+			JsonNode subtask = subtasks.get(x);
+			AccessExecutionVertex expectedSubtask = originalTask.getTaskVertices()[x];
+
+			Assert.assertEquals(x, subtask.get("subtask").asInt());
+			Assert.assertEquals(expectedSubtask.getCurrentExecutionAttempt().getAttemptNumber(), subtask.get("attempt").asInt());
+			Assert.assertEquals(expectedSubtask.getCurrentAssignedResourceLocation().getHostname(), subtask.get("host").asText());
+
+			ArchivedJobGenerationUtils.compareStringifiedAccumulators(
+				expectedSubtask.getCurrentExecutionAttempt().getUserAccumulatorsStringified(),
+				(ArrayNode) subtask.get("user-accumulators"));
+		}
+	}
+}


[07/16] flink git commit: [FLINK-7531] Move Flink legacy rest handler to flink-runtime

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandler.java
new file mode 100644
index 0000000..3236062
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobExceptionsHandler.java
@@ -0,0 +1,136 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.util.ExceptionUtils;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the configuration of a job.
+ */
+public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String JOB_EXCEPTIONS_REST_PATH = "/jobs/:jobid/exceptions";
+
+	static final int MAX_NUMBER_EXCEPTION_TO_REPORT = 20;
+
+	public JobExceptionsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_EXCEPTIONS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createJobExceptionsJson(graph);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create job exceptions json.", e);
+				}
+			},
+			executor
+		);
+	}
+
+	/**
+	 * Archivist for the JobExceptionsHandler.
+	 */
+	public static class JobExceptionsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String json = createJobExceptionsJson(graph);
+			String path = JOB_EXCEPTIONS_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+
+	public static String createJobExceptionsJson(AccessExecutionGraph graph) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+
+		// most important is the root failure cause
+		ErrorInfo rootException = graph.getFailureCause();
+		if (rootException != null && !rootException.getExceptionAsString().equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
+			gen.writeStringField("root-exception", rootException.getExceptionAsString());
+			gen.writeNumberField("timestamp", rootException.getTimestamp());
+		}
+
+		// we additionally collect all exceptions (up to a limit) that occurred in the individual tasks
+		gen.writeArrayFieldStart("all-exceptions");
+
+		int numExceptionsSoFar = 0;
+		boolean truncated = false;
+
+		for (AccessExecutionVertex task : graph.getAllExecutionVertices()) {
+			String t = task.getFailureCauseAsString();
+			if (t != null && !t.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
+				if (numExceptionsSoFar >= MAX_NUMBER_EXCEPTION_TO_REPORT) {
+					truncated = true;
+					break;
+				}
+
+				TaskManagerLocation location = task.getCurrentAssignedResourceLocation();
+				String locationString = location != null ?
+						location.getFQDNHostname() + ':' + location.dataPort() : "(unassigned)";
+
+				gen.writeStartObject();
+				gen.writeStringField("exception", t);
+				gen.writeStringField("task", task.getTaskNameWithSubtaskIndex());
+				gen.writeStringField("location", locationString);
+				long timestamp = task.getStateTimestamp(ExecutionState.FAILED);
+				gen.writeNumberField("timestamp", timestamp == 0 ? -1 : timestamp);
+				gen.writeEndObject();
+				numExceptionsSoFar++;
+			}
+		}
+		gen.writeEndArray();
+
+		gen.writeBooleanField("truncated", truncated);
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
new file mode 100644
index 0000000..364af91
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
@@ -0,0 +1,87 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Returns the Job Manager's configuration.
+ */
+public class JobManagerConfigHandler extends AbstractJsonRequestHandler {
+
+	private static final String JOBMANAGER_CONFIG_REST_PATH = "/jobmanager/config";
+
+	private final Configuration config;
+
+	public JobManagerConfigHandler(Executor executor, Configuration config) {
+		super(executor);
+		this.config = config;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOBMANAGER_CONFIG_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					StringWriter writer = new StringWriter();
+					JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+					gen.writeStartArray();
+					for (String key : config.keySet()) {
+						gen.writeStartObject();
+						gen.writeStringField("key", key);
+
+						// Mask key values which contain sensitive information
+						if (key.toLowerCase().contains("password")) {
+							String value = config.getString(key, null);
+							if (value != null) {
+								value = "******";
+							}
+							gen.writeStringField("value", value);
+						} else {
+							gen.writeStringField("value", config.getString(key, null));
+						}
+						gen.writeEndObject();
+					}
+					gen.writeEndArray();
+
+					gen.close();
+					return writer.toString();
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not write configuration.", e);
+				}
+			},
+			executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandler.java
new file mode 100644
index 0000000..d9db1ff
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobPlanHandler.java
@@ -0,0 +1,66 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the JSON program plan of a job graph.
+ */
+public class JobPlanHandler extends AbstractExecutionGraphRequestHandler {
+
+	private static final String JOB_PLAN_REST_PATH = "/jobs/:jobid/plan";
+
+	public JobPlanHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_PLAN_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionGraph graph, Map<String, String> params) {
+		return CompletableFuture.completedFuture(graph.getJsonPlan());
+	}
+
+	/**
+	 * Archivist for the JobPlanHandler.
+	 */
+	public static class JobPlanJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			String path = JOB_PLAN_REST_PATH
+				.replace(":jobid", graph.getJobID().toString());
+			String json = graph.getJsonPlan();
+			return Collections.singletonList(new ArchivedJson(path, json));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandler.java
new file mode 100644
index 0000000..cc41a1c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobStoppingHandler.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler for the STOP request.
+ */
+public class JobStoppingHandler extends AbstractJsonRequestHandler {
+
+	private static final String JOB_STOPPING_REST_PATH = "/jobs/:jobid/stop";
+	private static final String JOB_STOPPING_YARN_REST_PATH = "/jobs/:jobid/yarn-stop";
+
+	private final Time timeout;
+
+	public JobStoppingHandler(Executor executor, Time timeout) {
+		super(executor);
+		this.timeout = Preconditions.checkNotNull(timeout);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_STOPPING_REST_PATH, JOB_STOPPING_YARN_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid")));
+					if (jobManagerGateway != null) {
+						jobManagerGateway.stopJob(jobId, timeout);
+						return "{}";
+					}
+					else {
+						throw new Exception("No connection to the leading JobManager.");
+					}
+				}
+				catch (Exception e) {
+					throw new FlinkFutureException("Failed to stop the job with id: "  + pathParams.get("jobid") + '.', e);
+				}
+			},
+			executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandler.java
new file mode 100644
index 0000000..9830ab4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexAccumulatorsHandler.java
@@ -0,0 +1,112 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the accummulators for a given vertex.
+ */
+public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String JOB_VERTEX_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/accumulators";
+
+	public JobVertexAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_VERTEX_ACCUMULATORS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createVertexAccumulatorsJson(jobVertex);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create job vertex accumulators json.", e);
+				}
+			},
+			executor);
+
+	}
+
+	/**
+	 * Archivist for JobVertexAccumulatorsHandler.
+	 */
+	public static class JobVertexAccumulatorsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				String json = createVertexAccumulatorsJson(task);
+				String path = JOB_VERTEX_ACCUMULATORS_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":vertexid", task.getJobVertexId().toString());
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createVertexAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		StringifiedAccumulatorResult[] accs = jobVertex.getAggregatedUserAccumulatorsStringified();
+
+		gen.writeStartObject();
+		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
+
+		gen.writeArrayFieldStart("user-accumulators");
+		for (StringifiedAccumulatorResult acc : accs) {
+			gen.writeStartObject();
+			gen.writeStringField("name", acc.getName());
+			gen.writeStringField("type", acc.getType());
+			gen.writeStringField("value", acc.getValue());
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandler.java
new file mode 100644
index 0000000..59bfc0b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexBackPressureHandler.java
@@ -0,0 +1,145 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker;
+import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Request handler that returns back pressure stats for a single job vertex and
+ * all its sub tasks.
+ */
+public class JobVertexBackPressureHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String JOB_VERTEX_BACKPRESSURE_REST_PATH = "/jobs/:jobid/vertices/:vertexid/backpressure";
+
+	/** Back pressure stats tracker. */
+	private final BackPressureStatsTracker backPressureStatsTracker;
+
+	/** Time after which stats are considered outdated. */
+	private final int refreshInterval;
+
+	public JobVertexBackPressureHandler(
+			ExecutionGraphHolder executionGraphHolder,
+			Executor executor,
+			BackPressureStatsTracker backPressureStatsTracker,
+			int refreshInterval) {
+
+		super(executionGraphHolder, executor);
+		this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker, "Stats tracker");
+		checkArgument(refreshInterval >= 0, "Negative timeout");
+		this.refreshInterval = refreshInterval;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_VERTEX_BACKPRESSURE_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(
+			AccessExecutionJobVertex accessJobVertex,
+			Map<String, String> params) {
+		if (accessJobVertex instanceof ArchivedExecutionJobVertex) {
+			return CompletableFuture.completedFuture("");
+		}
+		ExecutionJobVertex jobVertex = (ExecutionJobVertex) accessJobVertex;
+		try (StringWriter writer = new StringWriter();
+				JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
+
+			gen.writeStartObject();
+
+			Optional<OperatorBackPressureStats> statsOption = backPressureStatsTracker
+					.getOperatorBackPressureStats(jobVertex);
+
+			if (statsOption.isPresent()) {
+				OperatorBackPressureStats stats = statsOption.get();
+
+				// Check whether we need to refresh
+				if (refreshInterval <= System.currentTimeMillis() - stats.getEndTimestamp()) {
+					backPressureStatsTracker.triggerStackTraceSample(jobVertex);
+					gen.writeStringField("status", "deprecated");
+				} else {
+					gen.writeStringField("status", "ok");
+				}
+
+				gen.writeStringField("backpressure-level", getBackPressureLevel(stats.getMaxBackPressureRatio()));
+				gen.writeNumberField("end-timestamp", stats.getEndTimestamp());
+
+				// Sub tasks
+				gen.writeArrayFieldStart("subtasks");
+				int numSubTasks = stats.getNumberOfSubTasks();
+				for (int i = 0; i < numSubTasks; i++) {
+					double ratio = stats.getBackPressureRatio(i);
+
+					gen.writeStartObject();
+					gen.writeNumberField("subtask", i);
+					gen.writeStringField("backpressure-level", getBackPressureLevel(ratio));
+					gen.writeNumberField("ratio", ratio);
+					gen.writeEndObject();
+				}
+				gen.writeEndArray();
+			} else {
+				backPressureStatsTracker.triggerStackTraceSample(jobVertex);
+				gen.writeStringField("status", "deprecated");
+			}
+
+			gen.writeEndObject();
+			gen.close();
+
+			return CompletableFuture.completedFuture(writer.toString());
+		} catch (IOException e) {
+			return FutureUtils.completedExceptionally(e);
+		}
+	}
+
+	/**
+	 * Returns the back pressure level as a String.
+	 *
+	 * @param backPressureRatio Ratio of back pressures samples to total number of samples.
+	 *
+	 * @return Back pressure level ('no', 'low', or 'high')
+	 */
+	static String getBackPressureLevel(double backPressureRatio) {
+		if (backPressureRatio <= 0.10) {
+			return "ok";
+		} else if (backPressureRatio <= 0.5) {
+			return "low";
+		} else {
+			return "high";
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandler.java
new file mode 100644
index 0000000..3f0c77c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexDetailsHandler.java
@@ -0,0 +1,159 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * A request handler that provides the details of a job vertex, including id, name, parallelism,
+ * and the runtime and metrics of all its subtasks.
+ */
+public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String JOB_VERTEX_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid";
+
+	private final MetricFetcher fetcher;
+
+	public JobVertexDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
+		super(executionGraphHolder, executor);
+		this.fetcher = fetcher;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_VERTEX_DETAILS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createVertexDetailsJson(jobVertex, params.get("jobid"), fetcher);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not write the vertex details json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the JobVertexDetailsHandler.
+	 */
+	public static class JobVertexDetailsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				String json = createVertexDetailsJson(task, graph.getJobID().toString(), null);
+				String path = JOB_VERTEX_DETAILS_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":vertexid", task.getJobVertexId().toString());
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createVertexDetailsJson(
+			AccessExecutionJobVertex jobVertex,
+			String jobID,
+			@Nullable MetricFetcher fetcher) throws IOException {
+		final long now = System.currentTimeMillis();
+
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+
+		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
+		gen.writeStringField("name", jobVertex.getName());
+		gen.writeNumberField("parallelism", jobVertex.getParallelism());
+		gen.writeNumberField("now", now);
+
+		gen.writeArrayFieldStart("subtasks");
+		int num = 0;
+		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
+			final ExecutionState status = vertex.getExecutionState();
+
+			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
+			String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort();
+
+			long startTime = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
+			if (startTime == 0) {
+				startTime = -1;
+			}
+			long endTime = status.isTerminal() ? vertex.getStateTimestamp(status) : -1;
+			long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1;
+
+			gen.writeStartObject();
+			gen.writeNumberField("subtask", num);
+			gen.writeStringField("status", status.name());
+			gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
+			gen.writeStringField("host", locationString);
+			gen.writeNumberField("start-time", startTime);
+			gen.writeNumberField("end-time", endTime);
+			gen.writeNumberField("duration", duration);
+
+			MutableIOMetrics counts = new MutableIOMetrics();
+
+			counts.addIOMetrics(
+				vertex.getCurrentExecutionAttempt(),
+				fetcher,
+				jobID,
+				jobVertex.getJobVertexId().toString()
+			);
+
+			counts.writeIOMetricsAsJson(gen);
+
+			gen.writeEndObject();
+
+			num++;
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandler.java
new file mode 100644
index 0000000..fa4ab67
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobVertexTaskManagersHandler.java
@@ -0,0 +1,210 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * A request handler that provides the details of a job vertex, including id, name, and the
+ * runtime and metrics of all its subtasks aggregated by TaskManager.
+ */
+public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String JOB_VERTEX_TASKMANAGERS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/taskmanagers";
+
+	private final MetricFetcher fetcher;
+
+	public JobVertexTaskManagersHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
+		super(executionGraphHolder, executor);
+		this.fetcher = fetcher;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{JOB_VERTEX_TASKMANAGERS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createVertexDetailsByTaskManagerJson(jobVertex, params.get("jobid"), fetcher);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create TaskManager json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for JobVertexTaskManagersHandler.
+	 */
+	public static class JobVertexTaskManagersJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				String json = createVertexDetailsByTaskManagerJson(task, graph.getJobID().toString(), null);
+				String path = JOB_VERTEX_TASKMANAGERS_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":vertexid", task.getJobVertexId().toString());
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createVertexDetailsByTaskManagerJson(
+			AccessExecutionJobVertex jobVertex,
+			String jobID,
+			@Nullable MetricFetcher fetcher) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		// Build a map that groups tasks by TaskManager
+		Map<String, List<AccessExecutionVertex>> taskManagerVertices = new HashMap<>();
+
+		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
+			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
+			String taskManager = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort();
+
+			List<AccessExecutionVertex> vertices = taskManagerVertices.get(taskManager);
+
+			if (vertices == null) {
+				vertices = new ArrayList<>();
+				taskManagerVertices.put(taskManager, vertices);
+			}
+
+			vertices.add(vertex);
+		}
+
+		// Build JSON response
+		final long now = System.currentTimeMillis();
+
+		gen.writeStartObject();
+
+		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
+		gen.writeStringField("name", jobVertex.getName());
+		gen.writeNumberField("now", now);
+
+		gen.writeArrayFieldStart("taskmanagers");
+		for (Map.Entry<String, List<AccessExecutionVertex>> entry : taskManagerVertices.entrySet()) {
+			String host = entry.getKey();
+			List<AccessExecutionVertex> taskVertices = entry.getValue();
+
+			int[] tasksPerState = new int[ExecutionState.values().length];
+
+			long startTime = Long.MAX_VALUE;
+			long endTime = 0;
+			boolean allFinished = true;
+
+			MutableIOMetrics counts = new MutableIOMetrics();
+
+			for (AccessExecutionVertex vertex : taskVertices) {
+				final ExecutionState state = vertex.getExecutionState();
+				tasksPerState[state.ordinal()]++;
+
+				// take the earliest start time
+				long started = vertex.getStateTimestamp(ExecutionState.DEPLOYING);
+				if (started > 0) {
+					startTime = Math.min(startTime, started);
+				}
+
+				allFinished &= state.isTerminal();
+				endTime = Math.max(endTime, vertex.getStateTimestamp(state));
+
+				counts.addIOMetrics(
+					vertex.getCurrentExecutionAttempt(),
+					fetcher,
+					jobID,
+					jobVertex.getJobVertexId().toString());
+			}
+
+			long duration;
+			if (startTime < Long.MAX_VALUE) {
+				if (allFinished) {
+					duration = endTime - startTime;
+				}
+				else {
+					endTime = -1L;
+					duration = now - startTime;
+				}
+			}
+			else {
+				startTime = -1L;
+				endTime = -1L;
+				duration = -1L;
+			}
+
+			ExecutionState jobVertexState =
+				ExecutionJobVertex.getAggregateJobVertexState(tasksPerState, taskVertices.size());
+
+			gen.writeStartObject();
+
+			gen.writeStringField("host", host);
+			gen.writeStringField("status", jobVertexState.name());
+
+			gen.writeNumberField("start-time", startTime);
+			gen.writeNumberField("end-time", endTime);
+			gen.writeNumberField("duration", duration);
+
+			counts.writeIOMetricsAsJson(gen);
+
+			gen.writeObjectFieldStart("status-counts");
+			for (ExecutionState state : ExecutionState.values()) {
+				gen.writeNumberField(state.name(), tasksPerState[state.ordinal()]);
+			}
+			gen.writeEndObject();
+
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JsonFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JsonFactory.java
new file mode 100644
index 0000000..e0629f3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JsonFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rest.handler.legacy;
+
+/**
+ * A holder for the singleton Jackson JSON factory. Since the Jackson's JSON factory object
+ * is a heavyweight object that is encouraged to be shared, we use a singleton instance across
+ * all request handlers.
+ */
+public class JsonFactory {
+
+	/** The singleton Jackson JSON factory. */
+	public static final com.fasterxml.jackson.core.JsonFactory JACKSON_FACTORY =
+			new com.fasterxml.jackson.core.JsonFactory();
+
+	// --------------------------------------------------------------------------------------------
+
+	private JsonFactory() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandler.java
new file mode 100644
index 0000000..d9fc899
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandler.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Base interface for all request handlers.
+ *
+ * <p>Most handlers will want to use the {@link AbstractJsonRequestHandler}
+ * as a starting point, which produces a valid HTTP response.
+ */
+public interface RequestHandler {
+
+	/**
+	 * Core method that handles the request and generates the response. The method needs to
+	 * respond with a full http response, including content-type, content-length, etc.
+	 *
+	 * <p>Exceptions may be throws and will be handled.
+	 *
+	 * @param pathParams The map of REST path parameters, decoded by the router.
+	 * @param queryParams The map of query parameters.
+	 * @param jobManagerGateway to talk to the JobManager.
+	 *
+	 * @return The full http response.
+	 */
+	CompletableFuture<FullHttpResponse> handleRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway);
+
+	/**
+	 * Returns an array of REST URL's under which this handler can be registered.
+	 *
+	 * @return array containing REST URL's under which this handler can be registered.
+	 */
+	String[] getPaths();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandlerException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandlerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandlerException.java
new file mode 100644
index 0000000..0dad905
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/RequestHandlerException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.rest.handler.legacy;
+
+/**
+ * Base class for request handler exceptions.
+ */
+public class RequestHandlerException extends Exception {
+
+	private static final long serialVersionUID = 7570352908725875886L;
+
+	public RequestHandlerException(String message) {
+		super(message);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandler.java
new file mode 100644
index 0000000..ff4fb46
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskCurrentAttemptDetailsHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler providing details about a single task execution attempt.
+ */
+public class SubtaskCurrentAttemptDetailsHandler extends SubtaskExecutionAttemptDetailsHandler {
+
+	public static final String SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum";
+
+	public SubtaskCurrentAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
+		super(executionGraphHolder, executor, fetcher);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionVertex vertex, Map<String, String> params) {
+		return handleRequest(vertex.getCurrentExecutionAttempt(), params);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandler.java
new file mode 100644
index 0000000..be4fe0b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptAccumulatorsHandler.java
@@ -0,0 +1,133 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Base class for request handlers whose response depends on a specific job vertex (defined
+ * via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
+ */
+public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskAttemptRequestHandler {
+
+	private static final String SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators";
+
+	public SubtaskExecutionAttemptAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createAttemptAccumulatorsJson(execAttempt);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create accumulator json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the SubtaskExecutionAttemptAccumulatorsHandler.
+	 */
+	public static class SubtaskExecutionAttemptAccumulatorsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				for (AccessExecutionVertex subtask : task.getTaskVertices()) {
+					String curAttemptJson = createAttemptAccumulatorsJson(subtask.getCurrentExecutionAttempt());
+					String curAttemptPath = SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH
+						.replace(":jobid", graph.getJobID().toString())
+						.replace(":vertexid", task.getJobVertexId().toString())
+						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
+						.replace(":attempt", String.valueOf(subtask.getCurrentExecutionAttempt().getAttemptNumber()));
+
+					archive.add(new ArchivedJson(curAttemptPath, curAttemptJson));
+
+					for (int x = 0; x < subtask.getCurrentExecutionAttempt().getAttemptNumber(); x++) {
+						AccessExecution attempt = subtask.getPriorExecutionAttempt(x);
+						String json = createAttemptAccumulatorsJson(attempt);
+						String path = SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH
+							.replace(":jobid", graph.getJobID().toString())
+							.replace(":vertexid", task.getJobVertexId().toString())
+							.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
+							.replace(":attempt", String.valueOf(attempt.getAttemptNumber()));
+						archive.add(new ArchivedJson(path, json));
+					}
+				}
+			}
+			return archive;
+		}
+	}
+
+	public static String createAttemptAccumulatorsJson(AccessExecution execAttempt) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		final StringifiedAccumulatorResult[] accs = execAttempt.getUserAccumulatorsStringified();
+
+		gen.writeStartObject();
+
+		gen.writeNumberField("subtask", execAttempt.getParallelSubtaskIndex());
+		gen.writeNumberField("attempt", execAttempt.getAttemptNumber());
+		gen.writeStringField("id", execAttempt.getAttemptId().toString());
+
+		gen.writeArrayFieldStart("user-accumulators");
+		for (StringifiedAccumulatorResult acc : accs) {
+			gen.writeStartObject();
+			gen.writeStringField("name", acc.getName());
+			gen.writeStringField("type", acc.getType());
+			gen.writeStringField("value", acc.getValue());
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandler.java
new file mode 100644
index 0000000..83a8793
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtaskExecutionAttemptDetailsHandler.java
@@ -0,0 +1,166 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecution;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
+import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.runtime.rest.handler.legacy.SubtaskCurrentAttemptDetailsHandler.SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH;
+
+/**
+ * Request handler providing details about a single task execution attempt.
+ */
+public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemptRequestHandler {
+
+	private static final String SUBTASK_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt";
+
+	private final MetricFetcher fetcher;
+
+	public SubtaskExecutionAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) {
+		super(executionGraphHolder, executor);
+		this.fetcher = fetcher;
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{SUBTASK_ATTEMPT_DETAILS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecution execAttempt, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createAttemptDetailsJson(execAttempt, params.get("jobid"), params.get("vertexid"), fetcher);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create attempt details json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the SubtaskExecutionAttemptDetailsHandler.
+	 */
+	public static class SubtaskExecutionAttemptDetailsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				for (AccessExecutionVertex subtask : task.getTaskVertices()) {
+					String curAttemptJson = createAttemptDetailsJson(subtask.getCurrentExecutionAttempt(), graph.getJobID().toString(), task.getJobVertexId().toString(), null);
+					String curAttemptPath1 = SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH
+						.replace(":jobid", graph.getJobID().toString())
+						.replace(":vertexid", task.getJobVertexId().toString())
+						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()));
+					String curAttemptPath2 = SUBTASK_ATTEMPT_DETAILS_REST_PATH
+						.replace(":jobid", graph.getJobID().toString())
+						.replace(":vertexid", task.getJobVertexId().toString())
+						.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
+						.replace(":attempt", String.valueOf(subtask.getCurrentExecutionAttempt().getAttemptNumber()));
+
+					archive.add(new ArchivedJson(curAttemptPath1, curAttemptJson));
+					archive.add(new ArchivedJson(curAttemptPath2, curAttemptJson));
+
+					for (int x = 0; x < subtask.getCurrentExecutionAttempt().getAttemptNumber(); x++) {
+						AccessExecution attempt = subtask.getPriorExecutionAttempt(x);
+						String json = createAttemptDetailsJson(attempt, graph.getJobID().toString(), task.getJobVertexId().toString(), null);
+						String path = SUBTASK_ATTEMPT_DETAILS_REST_PATH
+							.replace(":jobid", graph.getJobID().toString())
+							.replace(":vertexid", task.getJobVertexId().toString())
+							.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
+							.replace(":attempt", String.valueOf(attempt.getAttemptNumber()));
+						archive.add(new ArchivedJson(path, json));
+					}
+				}
+			}
+			return archive;
+		}
+	}
+
+	public static String createAttemptDetailsJson(
+			AccessExecution execAttempt,
+			String jobID,
+			String vertexID,
+			@Nullable MetricFetcher fetcher) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		final ExecutionState status = execAttempt.getState();
+		final long now = System.currentTimeMillis();
+
+		TaskManagerLocation location = execAttempt.getAssignedResourceLocation();
+		String locationString = location == null ? "(unassigned)" : location.getHostname();
+
+		long startTime = execAttempt.getStateTimestamp(ExecutionState.DEPLOYING);
+		if (startTime == 0) {
+			startTime = -1;
+		}
+		long endTime = status.isTerminal() ? execAttempt.getStateTimestamp(status) : -1;
+		long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1;
+
+		gen.writeStartObject();
+		gen.writeNumberField("subtask", execAttempt.getParallelSubtaskIndex());
+		gen.writeStringField("status", status.name());
+		gen.writeNumberField("attempt", execAttempt.getAttemptNumber());
+		gen.writeStringField("host", locationString);
+		gen.writeNumberField("start-time", startTime);
+		gen.writeNumberField("end-time", endTime);
+		gen.writeNumberField("duration", duration);
+
+		MutableIOMetrics counts = new MutableIOMetrics();
+
+		counts.addIOMetrics(
+			execAttempt,
+			fetcher,
+			jobID,
+			vertexID
+		);
+
+		counts.writeIOMetricsAsJson(gen);
+
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandler.java
new file mode 100644
index 0000000..6d0757d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksAllAccumulatorsHandler.java
@@ -0,0 +1,130 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the accumulators for all subtasks of job vertex.
+ */
+public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String SUBTASKS_ALL_ACCUMULATORS_REST_PATH = 	"/jobs/:jobid/vertices/:vertexid/subtasks/accumulators";
+
+	public SubtasksAllAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{SUBTASKS_ALL_ACCUMULATORS_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createSubtasksAccumulatorsJson(jobVertex);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not create subtasks accumulator json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the SubtasksAllAccumulatorsHandler.
+	 */
+	public static class SubtasksAllAccumulatorsJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				String json = createSubtasksAccumulatorsJson(task);
+				String path = SUBTASKS_ALL_ACCUMULATORS_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":vertexid", task.getJobVertexId().toString());
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createSubtasksAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
+		gen.writeNumberField("parallelism", jobVertex.getParallelism());
+
+		gen.writeArrayFieldStart("subtasks");
+
+		int num = 0;
+		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
+
+			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
+			String locationString = location == null ? "(unassigned)" : location.getHostname();
+
+			gen.writeStartObject();
+
+			gen.writeNumberField("subtask", num++);
+			gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
+			gen.writeStringField("host", locationString);
+
+			StringifiedAccumulatorResult[] accs = vertex.getCurrentExecutionAttempt().getUserAccumulatorsStringified();
+			gen.writeArrayFieldStart("user-accumulators");
+			for (StringifiedAccumulatorResult acc : accs) {
+				gen.writeStartObject();
+				gen.writeStringField("name", acc.getName());
+				gen.writeStringField("type", acc.getType());
+				gen.writeStringField("value", acc.getValue());
+				gen.writeEndObject();
+			}
+			gen.writeEndArray();
+
+			gen.writeEndObject();
+		}
+		gen.writeEndArray();
+
+		gen.writeEndObject();
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandler.java
new file mode 100644
index 0000000..13fdc16
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/SubtasksTimesHandler.java
@@ -0,0 +1,140 @@
+/*
+ * 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.rest.handler.legacy;
+
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * Request handler that returns the state transition timestamps for all subtasks, plus their
+ * location and duration.
+ */
+public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
+
+	private static final String SUBTASK_TIMES_REST_PATH = 	"/jobs/:jobid/vertices/:vertexid/subtasktimes";
+
+	public SubtasksTimesHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) {
+		super(executionGraphHolder, executor);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{SUBTASK_TIMES_REST_PATH};
+	}
+
+	@Override
+	public CompletableFuture<String> handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) {
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					return createSubtaskTimesJson(jobVertex);
+				} catch (IOException e) {
+					throw new FlinkFutureException("Could not write subtask time json.", e);
+				}
+			},
+			executor);
+	}
+
+	/**
+	 * Archivist for the SubtasksTimesHandler.
+	 */
+	public static class SubtasksTimesJsonArchivist implements JsonArchivist {
+
+		@Override
+		public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
+			List<ArchivedJson> archive = new ArrayList<>();
+			for (AccessExecutionJobVertex task : graph.getAllVertices().values()) {
+				String json = createSubtaskTimesJson(task);
+				String path = SUBTASK_TIMES_REST_PATH
+					.replace(":jobid", graph.getJobID().toString())
+					.replace(":vertexid", task.getJobVertexId().toString());
+				archive.add(new ArchivedJson(path, json));
+			}
+			return archive;
+		}
+	}
+
+	public static String createSubtaskTimesJson(AccessExecutionJobVertex jobVertex) throws IOException {
+		final long now = System.currentTimeMillis();
+
+		StringWriter writer = new StringWriter();
+		JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+		gen.writeStartObject();
+
+		gen.writeStringField("id", jobVertex.getJobVertexId().toString());
+		gen.writeStringField("name", jobVertex.getName());
+		gen.writeNumberField("now", now);
+
+		gen.writeArrayFieldStart("subtasks");
+
+		int num = 0;
+		for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
+
+			long[] timestamps = vertex.getCurrentExecutionAttempt().getStateTimestamps();
+			ExecutionState status = vertex.getExecutionState();
+
+			long scheduledTime = timestamps[ExecutionState.SCHEDULED.ordinal()];
+
+			long start = scheduledTime > 0 ? scheduledTime : -1;
+			long end = status.isTerminal() ? timestamps[status.ordinal()] : now;
+			long duration = start >= 0 ? end - start : -1L;
+
+			gen.writeStartObject();
+			gen.writeNumberField("subtask", num++);
+
+			TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
+			String locationString = location == null ? "(unassigned)" : location.getHostname();
+			gen.writeStringField("host", locationString);
+
+			gen.writeNumberField("duration", duration);
+
+			gen.writeObjectFieldStart("timestamps");
+			for (ExecutionState state : ExecutionState.values()) {
+				gen.writeNumberField(state.name(), timestamps[state.ordinal()]);
+			}
+			gen.writeEndObject();
+
+			gen.writeEndObject();
+		}
+
+		gen.writeEndArray();
+		gen.writeEndObject();
+
+		gen.close();
+		return writer.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc019a9/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java
new file mode 100644
index 0000000..718657e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java
@@ -0,0 +1,335 @@
+/*
+ * 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.rest.handler.legacy;
+
+/*****************************************************************************
+ * This code is based on the "HttpStaticFileServerHandler" from the
+ * Netty project's HTTP server example.
+ *
+ * See http://netty.io and
+ * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
+ *****************************************************************************/
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobCache;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.blob.BlobView;
+import org.apache.flink.runtime.concurrent.FlinkFutureException;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmaster.JobManagerGateway;
+import org.apache.flink.runtime.rest.handler.RedirectHandler;
+import org.apache.flink.runtime.rest.handler.WebHandler;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.nio.channels.FileChannel;
+import java.util.HashMap;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Request handler that returns the TaskManager log/out files.
+ *
+ * <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
+ * example.</p>
+ */
+@ChannelHandler.Sharable
+public class TaskManagerLogHandler extends RedirectHandler<JobManagerGateway> implements WebHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerLogHandler.class);
+
+	private static final String TASKMANAGER_LOG_REST_PATH = "/taskmanagers/:taskmanagerid/log";
+	private static final String TASKMANAGER_OUT_REST_PATH = "/taskmanagers/:taskmanagerid/stdout";
+
+	/** Keep track of last transmitted log, to clean up old ones. */
+	private final HashMap<String, BlobKey> lastSubmittedLog = new HashMap<>();
+	private final HashMap<String, BlobKey> lastSubmittedStdout = new HashMap<>();
+
+	/** Keep track of request status, prevents multiple log requests for a single TM running concurrently. */
+	private final ConcurrentHashMap<String, Boolean> lastRequestPending = new ConcurrentHashMap<>();
+	private final Configuration config;
+
+	/** Future of the blob cache. */
+	private CompletableFuture<BlobCache> cache;
+
+	/** Indicates which log file should be displayed. */
+	private FileMode fileMode;
+
+	private final Executor executor;
+
+	private final BlobView blobView;
+
+	/** Used to control whether this handler serves the .log or .out file. */
+	public enum FileMode {
+		LOG,
+		STDOUT
+	}
+
+	public TaskManagerLogHandler(
+		GatewayRetriever<JobManagerGateway> retriever,
+		Executor executor,
+		CompletableFuture<String> localJobManagerAddressPromise,
+		Time timeout,
+		FileMode fileMode,
+		Configuration config,
+		BlobView blobView) {
+		super(localJobManagerAddressPromise, retriever, timeout);
+
+		this.executor = checkNotNull(executor);
+		this.config = config;
+		this.fileMode = fileMode;
+
+		this.blobView = Preconditions.checkNotNull(blobView, "blobView");
+	}
+
+	@Override
+	public String[] getPaths() {
+		switch (fileMode) {
+			case LOG:
+				return new String[]{TASKMANAGER_LOG_REST_PATH};
+			case STDOUT:
+			default:
+				return new String[]{TASKMANAGER_OUT_REST_PATH};
+		}
+	}
+
+	/**
+	 * Response when running with leading JobManager.
+	 */
+	@Override
+	protected void respondAsLeader(final ChannelHandlerContext ctx, final Routed routed, final JobManagerGateway jobManagerGateway) {
+		if (cache == null) {
+			CompletableFuture<Integer> blobPortFuture = jobManagerGateway.requestBlobServerPort(timeout);
+			cache = blobPortFuture.thenApplyAsync(
+				(Integer port) -> {
+					try {
+						return new BlobCache(new InetSocketAddress(jobManagerGateway.getHostname(), port), config, blobView);
+					} catch (IOException e) {
+						throw new FlinkFutureException("Could not create BlobCache.", e);
+					}
+				},
+				executor);
+		}
+
+		final String taskManagerID = routed.pathParams().get(TaskManagersHandler.TASK_MANAGER_ID_KEY);
+		final HttpRequest request = routed.request();
+
+		//fetch TaskManager logs if no other process is currently doing it
+		if (lastRequestPending.putIfAbsent(taskManagerID, true) == null) {
+			try {
+				InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(taskManagerID));
+				CompletableFuture<Optional<Instance>> taskManagerFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout);
+
+				CompletableFuture<BlobKey> blobKeyFuture = taskManagerFuture.thenCompose(
+					(Optional<Instance> optTMInstance) -> {
+						Instance taskManagerInstance = optTMInstance.orElseThrow(
+							() -> new FlinkFutureException("Could not find instance with " + instanceID + '.'));
+						switch (fileMode) {
+							case LOG:
+								return taskManagerInstance.getTaskManagerGateway().requestTaskManagerLog(timeout);
+							case STDOUT:
+							default:
+								return taskManagerInstance.getTaskManagerGateway().requestTaskManagerStdout(timeout);
+						}
+					}
+				);
+
+				CompletableFuture<String> logPathFuture = blobKeyFuture
+					.thenCombineAsync(
+						cache,
+						(blobKey, blobCache) -> {
+							//delete previous log file, if it is different than the current one
+							HashMap<String, BlobKey> lastSubmittedFile = fileMode == FileMode.LOG ? lastSubmittedLog : lastSubmittedStdout;
+							if (lastSubmittedFile.containsKey(taskManagerID)) {
+								if (!Objects.equals(blobKey, lastSubmittedFile.get(taskManagerID))) {
+									try {
+										blobCache.deleteGlobal(lastSubmittedFile.get(taskManagerID));
+									} catch (IOException e) {
+										throw new FlinkFutureException("Could not delete file for " + taskManagerID + '.', e);
+									}
+									lastSubmittedFile.put(taskManagerID, blobKey);
+								}
+							} else {
+								lastSubmittedFile.put(taskManagerID, blobKey);
+							}
+							try {
+								return blobCache.getFile(blobKey).getAbsolutePath();
+							} catch (IOException e) {
+								throw new FlinkFutureException("Could not retrieve blob for " + blobKey + '.', e);
+							}
+						},
+						executor);
+
+				logPathFuture.exceptionally(
+					failure -> {
+						display(ctx, request, "Fetching TaskManager log failed.");
+						LOG.error("Fetching TaskManager log failed.", failure);
+						lastRequestPending.remove(taskManagerID);
+
+						return null;
+					});
+
+				logPathFuture.thenAccept(
+					filePath -> {
+						File file = new File(filePath);
+						final RandomAccessFile raf;
+						try {
+							raf = new RandomAccessFile(file, "r");
+						} catch (FileNotFoundException e) {
+							display(ctx, request, "Displaying TaskManager log failed.");
+							LOG.error("Displaying TaskManager log failed.", e);
+
+							return;
+						}
+						long fileLength;
+						try {
+							fileLength = raf.length();
+						} catch (IOException ioe) {
+							display(ctx, request, "Displaying TaskManager log failed.");
+							LOG.error("Displaying TaskManager log failed.", ioe);
+							try {
+								raf.close();
+							} catch (IOException e) {
+								LOG.error("Could not close random access file.", e);
+							}
+
+							return;
+						}
+						final FileChannel fc = raf.getChannel();
+
+						HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+						response.headers().set(CONTENT_TYPE, "text/plain");
+
+						if (HttpHeaders.isKeepAlive(request)) {
+							response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+						}
+						HttpHeaders.setContentLength(response, fileLength);
+
+						// write the initial line and the header.
+						ctx.write(response);
+
+						// write the content.
+						ChannelFuture lastContentFuture;
+						final GenericFutureListener<Future<? super Void>> completionListener = future -> {
+							lastRequestPending.remove(taskManagerID);
+							fc.close();
+							raf.close();
+						};
+						if (ctx.pipeline().get(SslHandler.class) == null) {
+							ctx.write(
+								new DefaultFileRegion(fc, 0, fileLength), ctx.newProgressivePromise())
+									.addListener(completionListener);
+							lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+
+						} else {
+							try {
+								lastContentFuture = ctx.writeAndFlush(
+									new HttpChunkedInput(new ChunkedFile(raf, 0, fileLength, 8192)),
+									ctx.newProgressivePromise())
+									.addListener(completionListener);
+							} catch (IOException e) {
+								display(ctx, request, "Displaying TaskManager log failed.");
+								LOG.warn("Could not write http data.", e);
+
+								return;
+							}
+							// HttpChunkedInput will write the end marker (LastHttpContent) for us.
+						}
+
+						// close the connection, if no keep-alive is needed
+						if (!HttpHeaders.isKeepAlive(request)) {
+							lastContentFuture.addListener(ChannelFutureListener.CLOSE);
+						}
+					});
+			} catch (Exception e) {
+				display(ctx, request, "Error: " + e.getMessage());
+				LOG.error("Fetching TaskManager log failed.", e);
+				lastRequestPending.remove(taskManagerID);
+			}
+		} else {
+			display(ctx, request, "loading...");
+		}
+	}
+
+	private void display(ChannelHandlerContext ctx, HttpRequest request, String message) {
+		HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+		response.headers().set(CONTENT_TYPE, "text/plain");
+
+		if (HttpHeaders.isKeepAlive(request)) {
+			response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+		}
+
+		byte[] buf = message.getBytes(ConfigConstants.DEFAULT_CHARSET);
+
+		ByteBuf b = Unpooled.copiedBuffer(buf);
+
+		HttpHeaders.setContentLength(response, buf.length);
+
+		// write the initial line and the header.
+		ctx.write(response);
+
+		ctx.write(b);
+
+		ChannelFuture lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+
+		// close the connection, if no keep-alive is needed
+		if (!HttpHeaders.isKeepAlive(request)) {
+			lastContentFuture.addListener(ChannelFutureListener.CLOSE);
+		}
+	}
+}