You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2018/06/21 17:59:09 UTC

[1/5] flink git commit: [FLINK-9467][metrics][WebUI] Fix watermark display

Repository: flink
Updated Branches:
  refs/heads/release-1.5 0796ac7e2 -> e0265062e


[FLINK-9467][metrics][WebUI] Fix watermark display

This closes #6152.


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

Branch: refs/heads/release-1.5
Commit: 986a377b736c561141b66d8d1dbf207cc70317fb
Parents: 0796ac7
Author: zentol <ch...@apache.org>
Authored: Tue Jun 5 13:51:08 2018 +0200
Committer: zentol <ch...@apache.org>
Committed: Thu Jun 21 16:41:25 2018 +0200

----------------------------------------------------------------------
 docs/monitoring/metrics.md                                   | 8 ++++----
 .../web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee  | 4 ++--
 .../flink/streaming/runtime/tasks/OneInputStreamTask.java    | 1 +
 .../flink/streaming/runtime/tasks/TwoInputStreamTask.java    | 1 +
 4 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/986a377b/docs/monitoring/metrics.md
----------------------------------------------------------------------
diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md
index 49d7ba8..7fe5e7a 100644
--- a/docs/monitoring/metrics.md
+++ b/docs/monitoring/metrics.md
@@ -1228,7 +1228,7 @@ Thus, in order to infer the metric identifier:
       <td>Meter</td>
     </tr>
     <tr>
-      <th rowspan="5"><strong>Task/Operator</strong></th>
+      <th rowspan="6"><strong>Task/Operator</strong></th>
       <td>numRecordsIn</td>
       <td>The total number of records this operator/task has received.</td>
       <td>Counter</td>
@@ -1254,15 +1254,15 @@ Thus, in order to infer the metric identifier:
       <td>Counter</td>
     </tr>
     <tr>
-      <th rowspan="5"><strong>Operator</strong></th>
       <td>currentInputWatermark</td>
       <td>
-        The last watermark this operator has received (in milliseconds).
-        <p><strong>Note:</strong> For operators with 2 inputs this is the minimum of the last received watermarks.</p>
+        The last watermark this operator/tasks has received (in milliseconds).
+        <p><strong>Note:</strong> For operators/tasks with 2 inputs this is the minimum of the last received watermarks.</p>
       </td>
       <td>Gauge</td>
     </tr>
     <tr>
+      <th rowspan="4"><strong>Operator</strong></th>
       <td>currentInput1Watermark</td>
       <td>
         The last watermark this operator has received in its first input (in milliseconds).

http://git-wip-us.apache.org/repos/asf/flink/blob/986a377b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee
index 517b00c..ed2b22d 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee
@@ -107,13 +107,13 @@ angular.module('flinkApp')
       jid = $scope.job.jid
 
       # Request metrics for each subtask
-      metricIds = (i + ".currentLowWatermark" for i in [0..node.parallelism - 1])
+      metricIds = (i + ".currentInputWatermark" for i in [0..node.parallelism - 1])
       MetricsService.getMetrics(jid, node.id, metricIds).then (metrics) ->
         minValue = NaN
         watermarks = {}
 
         for key, value of metrics.values
-          subtaskIndex = key.replace('.currentLowWatermark', '')
+          subtaskIndex = key.replace('.currentInputWatermark', '')
           watermarks[subtaskIndex] = value
 
           if (isNaN(minValue) || value < minValue)

http://git-wip-us.apache.org/repos/asf/flink/blob/986a377b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
index 26088e4..43eab24 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
@@ -93,6 +93,7 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
 					inputWatermarkGauge);
 		}
 		headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge);
+		getEnvironment().getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/986a377b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
index bd878f6..93a5675 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
@@ -105,6 +105,7 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputS
 		headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge);
 		headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_1_WATERMARK, input1WatermarkGauge);
 		headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_2_WATERMARK, input2WatermarkGauge);
+		getEnvironment().getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge);
 	}
 
 	@Override


[5/5] flink git commit: [FLINK-9599][rest] Implement generic mechanism to access uploaded files

Posted by ch...@apache.org.
[FLINK-9599][rest] Implement generic mechanism to access uploaded files

This closes #6178.


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

Branch: refs/heads/release-1.5
Commit: e0265062e95df7c4cbcbc78c62c735174549cd4b
Parents: 4b2332a
Author: zentol <ch...@apache.org>
Authored: Mon Jun 18 10:54:42 2018 +0200
Committer: zentol <ch...@apache.org>
Committed: Thu Jun 21 17:49:59 2018 +0200

----------------------------------------------------------------------
 .../webmonitor/handlers/JarUploadHandler.java   |  36 +-
 .../webmonitor/handlers/JarUploadHeaders.java   |  13 +-
 .../handlers/JarUploadHandlerTest.java          |  18 +-
 flink-runtime/pom.xml                           |   7 +
 .../flink/runtime/rest/AbstractHandler.java     |  97 ++--
 .../flink/runtime/rest/FileUploadHandler.java   | 142 +++--
 .../rest/handler/AbstractRestHandler.java       |  22 +-
 .../flink/runtime/rest/handler/FileUploads.java |  98 ++++
 .../runtime/rest/handler/HandlerRequest.java    |  17 +-
 .../flink/runtime/rest/messages/FileUpload.java |  44 --
 .../messages/UntypedResponseMessageHeaders.java |   9 +
 .../flink/runtime/rest/AbstractHandlerTest.java | 191 +++++++
 .../runtime/rest/FileUploadHandlerTest.java     | 539 +++++++++++++++++++
 .../runtime/rest/RestServerEndpointITCase.java  |  38 +-
 .../runtime/rest/handler/FileUploadsTest.java   | 112 ++++
 15 files changed, 1199 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/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 c2f16a7..a1ef82b 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
@@ -23,7 +23,7 @@ import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
 import org.apache.flink.runtime.rest.handler.HandlerRequest;
 import org.apache.flink.runtime.rest.handler.RestHandlerException;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.FileUpload;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.runtime.rest.messages.MessageHeaders;
 import org.apache.flink.runtime.webmonitor.RestfulGateway;
 import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
@@ -35,6 +35,7 @@ import javax.annotation.Nonnull;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
@@ -46,7 +47,7 @@ import static java.util.Objects.requireNonNull;
  * Handles .jar file uploads.
  */
 public class JarUploadHandler extends
-		AbstractRestHandler<RestfulGateway, FileUpload, JarUploadResponseBody, EmptyMessageParameters> {
+		AbstractRestHandler<RestfulGateway, EmptyRequestBody, JarUploadResponseBody, EmptyMessageParameters> {
 
 	private final Path jarDir;
 
@@ -57,7 +58,7 @@ public class JarUploadHandler extends
 			final GatewayRetriever<? extends RestfulGateway> leaderRetriever,
 			final Time timeout,
 			final Map<String, String> responseHeaders,
-			final MessageHeaders<FileUpload, JarUploadResponseBody, EmptyMessageParameters> messageHeaders,
+			final MessageHeaders<EmptyRequestBody, JarUploadResponseBody, EmptyMessageParameters> messageHeaders,
 			final Path jarDir,
 			final Executor executor) {
 		super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders);
@@ -67,41 +68,34 @@ public class JarUploadHandler extends
 
 	@Override
 	protected CompletableFuture<JarUploadResponseBody> handleRequest(
-			@Nonnull final HandlerRequest<FileUpload, EmptyMessageParameters> request,
+			@Nonnull final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request,
 			@Nonnull final RestfulGateway gateway) throws RestHandlerException {
-
-		final FileUpload fileUpload = request.getRequestBody();
+		Collection<Path> uploadedFiles = request.getUploadedFiles();
+		if (uploadedFiles.size() != 1) {
+			throw new RestHandlerException("Exactly 1 file must be sent, received " + uploadedFiles.size() + '.', HttpResponseStatus.BAD_REQUEST);
+		}
+		final Path fileUpload = uploadedFiles.iterator().next();
 		return CompletableFuture.supplyAsync(() -> {
-			if (!fileUpload.getPath().getFileName().toString().endsWith(".jar")) {
-				deleteUploadedFile(fileUpload);
+			if (!fileUpload.getFileName().toString().endsWith(".jar")) {
 				throw new CompletionException(new RestHandlerException(
 					"Only Jar files are allowed.",
 					HttpResponseStatus.BAD_REQUEST));
 			} else {
-				final Path destination = jarDir.resolve(fileUpload.getPath().getFileName());
+				final Path destination = jarDir.resolve(fileUpload.getFileName());
 				try {
-					Files.move(fileUpload.getPath(), destination);
+					Files.move(fileUpload, destination);
 				} catch (IOException e) {
-					deleteUploadedFile(fileUpload);
 					throw new CompletionException(new RestHandlerException(
 						String.format("Could not move uploaded jar file [%s] to [%s].",
-							fileUpload.getPath(),
+							fileUpload,
 							destination),
 						HttpResponseStatus.INTERNAL_SERVER_ERROR,
 						e));
 				}
-				return new JarUploadResponseBody(fileUpload.getPath()
+				return new JarUploadResponseBody(fileUpload
 					.normalize()
 					.toString());
 			}
 		}, executor);
 	}
-
-	private void deleteUploadedFile(final FileUpload fileUpload) {
-		try {
-			Files.delete(fileUpload.getPath());
-		} catch (IOException e) {
-			log.error("Failed to delete file {}.", fileUpload.getPath(), e);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java
index 9408dba..d969b4fc 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.rest.HttpMethodWrapper;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.FileUpload;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.runtime.rest.messages.MessageHeaders;
 
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
@@ -28,7 +28,7 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt
 /**
  * {@link MessageHeaders} for uploading jars.
  */
-public final class JarUploadHeaders implements MessageHeaders<FileUpload, JarUploadResponseBody, EmptyMessageParameters> {
+public final class JarUploadHeaders implements MessageHeaders<EmptyRequestBody, JarUploadResponseBody, EmptyMessageParameters> {
 
 	public static final String URL = "/jars/upload";
 	private static final JarUploadHeaders INSTANCE = new JarUploadHeaders();
@@ -46,8 +46,8 @@ public final class JarUploadHeaders implements MessageHeaders<FileUpload, JarUpl
 	}
 
 	@Override
-	public Class<FileUpload> getRequestClass() {
-		return FileUpload.class;
+	public Class<EmptyRequestBody> getRequestClass() {
+		return EmptyRequestBody.class;
 	}
 
 	@Override
@@ -75,4 +75,9 @@ public final class JarUploadHeaders implements MessageHeaders<FileUpload, JarUpl
 			" header is set to \"application/x-java-archive\", as some http libraries do not add the header by default.\n" +
 			"Using 'curl' you can upload a jar via 'curl -X POST -H \"Expect:\" -F \"jarfile=#path/to/flink-job.jar\" http://hostname:port" + URL + "'.";
 	}
+
+	@Override
+	public boolean acceptsFileUploads() {
+		return true;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java
index 2ce75e7..812d4c6 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java
@@ -25,7 +25,7 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest;
 import org.apache.flink.runtime.rest.handler.HandlerRequestException;
 import org.apache.flink.runtime.rest.handler.RestHandlerException;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.FileUpload;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.TestLogger;
 
@@ -38,6 +38,7 @@ import org.junit.rules.TemporaryFolder;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Collections;
@@ -83,7 +84,7 @@ public class JarUploadHandlerTest extends TestLogger {
 	@Test
 	public void testRejectNonJarFiles() throws Exception {
 		final Path uploadedFile = Files.createFile(jarDir.resolve("katrin.png"));
-		final HandlerRequest<FileUpload, EmptyMessageParameters> request = createRequest(uploadedFile);
+		final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request = createRequest(uploadedFile);
 
 		try {
 			jarUploadHandler.handleRequest(request, mockDispatcherGateway).get();
@@ -99,7 +100,7 @@ public class JarUploadHandlerTest extends TestLogger {
 	@Test
 	public void testUploadJar() throws Exception {
 		final Path uploadedFile = Files.createFile(jarDir.resolve("Kafka010Example.jar"));
-		final HandlerRequest<FileUpload, EmptyMessageParameters> request = createRequest(uploadedFile);
+		final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request = createRequest(uploadedFile);
 
 		final JarUploadResponseBody jarUploadResponseBody = jarUploadHandler.handleRequest(request, mockDispatcherGateway).get();
 		assertThat(jarUploadResponseBody.getStatus(), equalTo(JarUploadResponseBody.UploadStatus.success));
@@ -109,7 +110,7 @@ public class JarUploadHandlerTest extends TestLogger {
 	@Test
 	public void testFailedUpload() throws Exception {
 		final Path uploadedFile = jarDir.resolve("Kafka010Example.jar");
-		final HandlerRequest<FileUpload, EmptyMessageParameters> request = createRequest(uploadedFile);
+		final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request = createRequest(uploadedFile);
 
 		try {
 			jarUploadHandler.handleRequest(request, mockDispatcherGateway).get();
@@ -123,12 +124,13 @@ public class JarUploadHandlerTest extends TestLogger {
 		}
 	}
 
-	private static HandlerRequest<FileUpload, EmptyMessageParameters> createRequest(
-			final Path uploadedFile) throws HandlerRequestException {
+	private static HandlerRequest<EmptyRequestBody, EmptyMessageParameters> createRequest(
+			final Path uploadedFile) throws HandlerRequestException, IOException {
 		return new HandlerRequest<>(
-			new FileUpload(uploadedFile),
+			EmptyRequestBody.getInstance(),
 			EmptyMessageParameters.getInstance(),
 			Collections.emptyMap(),
-			Collections.emptyMap());
+			Collections.emptyMap(),
+			Collections.singleton(uploadedFile));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 9084cfd..9493e9d 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -291,6 +291,13 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>com.squareup.okhttp3</groupId>
+			<artifactId>okhttp</artifactId>
+			<version>3.7.0</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
 			<groupId>com.typesafe.akka</groupId>
 			<artifactId>akka-testkit_${scala.binary.version}</artifactId>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java
index cb50a4f..41d242b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java
@@ -19,13 +19,13 @@
 package org.apache.flink.runtime.rest;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.rest.handler.FileUploads;
 import org.apache.flink.runtime.rest.handler.HandlerRequest;
 import org.apache.flink.runtime.rest.handler.HandlerRequestException;
 import org.apache.flink.runtime.rest.handler.RedirectHandler;
 import org.apache.flink.runtime.rest.handler.RestHandlerException;
 import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
 import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
-import org.apache.flink.runtime.rest.messages.FileUpload;
 import org.apache.flink.runtime.rest.messages.MessageParameters;
 import org.apache.flink.runtime.rest.messages.RequestBody;
 import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
@@ -50,7 +50,6 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnull;
 
-import java.nio.file.Path;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 
@@ -104,73 +103,73 @@ public abstract class AbstractHandler<T extends RestfulGateway, R extends Reques
 				return;
 			}
 
-			ByteBuf msgContent = ((FullHttpRequest) httpRequest).content();
+			final ByteBuf msgContent = ((FullHttpRequest) httpRequest).content();
 
-			R request;
-			if (isFileUpload()) {
-				final Path path = ctx.channel().attr(FileUploadHandler.UPLOADED_FILE).get();
-				if (path == null) {
+			try (FileUploads uploadedFiles = FileUploadHandler.getMultipartFileUploads(ctx)) {
+
+				if (!untypedResponseMessageHeaders.acceptsFileUploads() && !uploadedFiles.getUploadedFiles().isEmpty()) {
 					HandlerUtils.sendErrorResponse(
 						ctx,
 						httpRequest,
-						new ErrorResponseBody("Client did not upload a file."),
+						new ErrorResponseBody("File uploads not allowed."),
 						HttpResponseStatus.BAD_REQUEST,
 						responseHeaders);
 					return;
 				}
-				//noinspection unchecked
-				request = (R) new FileUpload(path);
-			} else if (msgContent.capacity() == 0) {
-				try {
-					request = MAPPER.readValue("{}", untypedResponseMessageHeaders.getRequestClass());
-				} catch (JsonParseException | JsonMappingException je) {
-					log.error("Request did not conform to expected format.", je);
-					HandlerUtils.sendErrorResponse(
-						ctx,
-						httpRequest,
-						new ErrorResponseBody("Bad request received."),
-						HttpResponseStatus.BAD_REQUEST,
-						responseHeaders);
-					return;
+
+				R request;
+				if (msgContent.capacity() == 0) {
+					try {
+						request = MAPPER.readValue("{}", untypedResponseMessageHeaders.getRequestClass());
+					} catch (JsonParseException | JsonMappingException je) {
+						log.error("Request did not conform to expected format.", je);
+						HandlerUtils.sendErrorResponse(
+							ctx,
+							httpRequest,
+							new ErrorResponseBody("Bad request received."),
+							HttpResponseStatus.BAD_REQUEST,
+							responseHeaders);
+						return;
+					}
+				} else {
+					try {
+						ByteBufInputStream in = new ByteBufInputStream(msgContent);
+						request = MAPPER.readValue(in, untypedResponseMessageHeaders.getRequestClass());
+					} catch (JsonParseException | JsonMappingException je) {
+						log.error("Failed to read request.", je);
+						HandlerUtils.sendErrorResponse(
+							ctx,
+							httpRequest,
+							new ErrorResponseBody(String.format("Request did not match expected format %s.", untypedResponseMessageHeaders.getRequestClass().getSimpleName())),
+							HttpResponseStatus.BAD_REQUEST,
+							responseHeaders);
+						return;
+					}
 				}
-			} else {
+
+				final HandlerRequest<R, M> handlerRequest;
+
 				try {
-					ByteBufInputStream in = new ByteBufInputStream(msgContent);
-					request = MAPPER.readValue(in, untypedResponseMessageHeaders.getRequestClass());
-				} catch (JsonParseException | JsonMappingException je) {
-					log.error("Failed to read request.", je);
+					handlerRequest = new HandlerRequest<>(request, untypedResponseMessageHeaders.getUnresolvedMessageParameters(), routed.pathParams(), routed.queryParams(), uploadedFiles.getUploadedFiles());
+				} catch (HandlerRequestException hre) {
+					log.error("Could not create the handler request.", hre);
+
 					HandlerUtils.sendErrorResponse(
 						ctx,
 						httpRequest,
-						new ErrorResponseBody(String.format("Request did not match expected format %s.", untypedResponseMessageHeaders.getRequestClass().getSimpleName())),
+						new ErrorResponseBody(String.format("Bad request, could not parse parameters: %s", hre.getMessage())),
 						HttpResponseStatus.BAD_REQUEST,
 						responseHeaders);
 					return;
 				}
-			}
-
-			final HandlerRequest<R, M> handlerRequest;
-
-			try {
-				handlerRequest = new HandlerRequest<>(request, untypedResponseMessageHeaders.getUnresolvedMessageParameters(), routed.pathParams(), routed.queryParams());
-			} catch (HandlerRequestException hre) {
-				log.error("Could not create the handler request.", hre);
 
-				HandlerUtils.sendErrorResponse(
+				respondToRequest(
 					ctx,
 					httpRequest,
-					new ErrorResponseBody(String.format("Bad request, could not parse parameters: %s", hre.getMessage())),
-					HttpResponseStatus.BAD_REQUEST,
-					responseHeaders);
-				return;
+					handlerRequest,
+					gateway);
 			}
 
-			respondToRequest(
-				ctx,
-				httpRequest,
-				handlerRequest,
-				gateway);
-
 		} catch (Throwable e) {
 			log.error("Request processing failed.", e);
 			HandlerUtils.sendErrorResponse(
@@ -182,10 +181,6 @@ public abstract class AbstractHandler<T extends RestfulGateway, R extends Reques
 		}
 	}
 
-	private boolean isFileUpload() {
-		return untypedResponseMessageHeaders.getRequestClass() == FileUpload.class;
-	}
-
 	/**
 	 * Respond to the given {@link HandlerRequest}.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
index 8854a1f..babc7c0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java
@@ -18,15 +18,24 @@
 
 package org.apache.flink.runtime.rest;
 
+import org.apache.flink.runtime.rest.handler.FileUploads;
+import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
+import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
+import org.apache.flink.util.FileUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPipeline;
 import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultLastHttpContent;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpContent;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+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.multipart.Attribute;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DiskFileUpload;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpDataFactory;
@@ -37,8 +46,13 @@ import org.apache.flink.shaded.netty4.io.netty.util.AttributeKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Optional;
 import java.util.UUID;
 
 import static java.util.Objects.requireNonNull;
@@ -52,7 +66,9 @@ public class FileUploadHandler extends SimpleChannelInboundHandler<HttpObject> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(FileUploadHandler.class);
 
-	static final AttributeKey<Path> UPLOADED_FILE = AttributeKey.valueOf("UPLOADED_FILE");
+	public static final String HTTP_ATTRIBUTE_REQUEST = "request";
+
+	private static final AttributeKey<FileUploads> UPLOADED_FILES = AttributeKey.valueOf("UPLOADED_FILES");
 
 	private static final HttpDataFactory DATA_FACTORY = new DefaultHttpDataFactory(true);
 
@@ -61,6 +77,8 @@ public class FileUploadHandler extends SimpleChannelInboundHandler<HttpObject> {
 	private HttpPostRequestDecoder currentHttpPostRequestDecoder;
 
 	private HttpRequest currentHttpRequest;
+	private byte[] currentJsonPayload;
+	private Path currentUploadDir;
 
 	public FileUploadHandler(final Path uploadDir) {
 		super(false);
@@ -70,52 +88,108 @@ public class FileUploadHandler extends SimpleChannelInboundHandler<HttpObject> {
 
 	@Override
 	protected void channelRead0(final ChannelHandlerContext ctx, final HttpObject msg) throws Exception {
-		if (msg instanceof HttpRequest) {
-			final HttpRequest httpRequest = (HttpRequest) msg;
-			if (httpRequest.getMethod().equals(HttpMethod.POST)) {
-				if (HttpPostRequestDecoder.isMultipart(httpRequest)) {
-					currentHttpPostRequestDecoder = new HttpPostRequestDecoder(DATA_FACTORY, httpRequest);
-					currentHttpRequest = httpRequest;
+		try {
+			if (msg instanceof HttpRequest) {
+				final HttpRequest httpRequest = (HttpRequest) msg;
+				LOG.trace("Received request. URL:{} Method:{}", httpRequest.getUri(), httpRequest.getMethod());
+				if (httpRequest.getMethod().equals(HttpMethod.POST)) {
+					if (HttpPostRequestDecoder.isMultipart(httpRequest)) {
+						currentHttpPostRequestDecoder = new HttpPostRequestDecoder(DATA_FACTORY, httpRequest);
+						currentHttpRequest = httpRequest;
+						currentUploadDir = Files.createDirectory(uploadDir.resolve(UUID.randomUUID().toString()));
+					} else {
+						ctx.fireChannelRead(msg);
+					}
 				} else {
 					ctx.fireChannelRead(msg);
 				}
+			} else if (msg instanceof HttpContent && currentHttpPostRequestDecoder != null) {
+				// make sure that we still have a upload dir in case that it got deleted in the meanwhile
+				RestServerEndpoint.createUploadDir(uploadDir, LOG);
+
+				final HttpContent httpContent = (HttpContent) msg;
+				currentHttpPostRequestDecoder.offer(httpContent);
+
+				while (currentHttpPostRequestDecoder.hasNext()) {
+					final InterfaceHttpData data = currentHttpPostRequestDecoder.next();
+					if (data.getHttpDataType() == InterfaceHttpData.HttpDataType.FileUpload) {
+						final DiskFileUpload fileUpload = (DiskFileUpload) data;
+						checkState(fileUpload.isCompleted());
+
+						final Path dest = currentUploadDir.resolve(fileUpload.getFilename());
+						fileUpload.renameTo(dest.toFile());
+					} else if (data.getHttpDataType() == InterfaceHttpData.HttpDataType.Attribute) {
+						final Attribute request = (Attribute) data;
+						// this could also be implemented by using the first found Attribute as the payload
+						if (data.getName().equals(HTTP_ATTRIBUTE_REQUEST)) {
+							currentJsonPayload = request.get();
+						} else {
+							handleError(ctx, "Received unknown attribute " + data.getName() + '.', HttpResponseStatus.BAD_REQUEST, null);
+							return;
+						}
+					}
+				}
+
+				if (httpContent instanceof LastHttpContent) {
+					ctx.channel().attr(UPLOADED_FILES).set(new FileUploads(currentUploadDir));
+					ctx.fireChannelRead(currentHttpRequest);
+					if (currentJsonPayload != null) {
+						// the following lines behave similar to httpContent#replace in netty 4.1
+						// the only difference is that the validateHeaders flag isn't preserved
+						// this shouldn't be a problem since we only copy existing headers
+						DefaultLastHttpContent newContent = new DefaultLastHttpContent(Unpooled.wrappedBuffer(currentJsonPayload), false);
+						newContent.trailingHeaders().set(((LastHttpContent) httpContent).trailingHeaders());
+						ctx.fireChannelRead(newContent);
+					} else {
+						ctx.fireChannelRead(httpContent);
+					}
+					reset();
+				}
 			} else {
 				ctx.fireChannelRead(msg);
 			}
-		} else if (msg instanceof HttpContent && currentHttpPostRequestDecoder != null) {
-			// make sure that we still have a upload dir in case that it got deleted in the meanwhile
-			RestServerEndpoint.createUploadDir(uploadDir, LOG);
-
-			final HttpContent httpContent = (HttpContent) msg;
-			currentHttpPostRequestDecoder.offer(httpContent);
-
-			while (currentHttpPostRequestDecoder.hasNext()) {
-				final InterfaceHttpData data = currentHttpPostRequestDecoder.next();
-				if (data.getHttpDataType() == InterfaceHttpData.HttpDataType.FileUpload) {
-					final DiskFileUpload fileUpload = (DiskFileUpload) data;
-					checkState(fileUpload.isCompleted());
-
-					final Path dest = uploadDir.resolve(Paths.get(UUID.randomUUID() +
-						"_" + fileUpload.getFilename()));
-					fileUpload.renameTo(dest.toFile());
-					ctx.channel().attr(UPLOADED_FILE).set(dest);
-				}
-				data.release();
-			}
+		} catch (Exception e) {
+			handleError(ctx, "File upload failed.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
+		}
+	}
 
-			if (httpContent instanceof LastHttpContent) {
-				ctx.fireChannelRead(currentHttpRequest);
-				ctx.fireChannelRead(httpContent);
-				reset();
+	private void handleError(ChannelHandlerContext ctx, String errorMessage, HttpResponseStatus responseStatus, @Nullable Throwable e) {
+		HttpRequest tmpRequest = currentHttpRequest;
+		deleteUploadedFiles();
+		reset();
+		LOG.warn(errorMessage, e);
+		HandlerUtils.sendErrorResponse(
+			ctx,
+			tmpRequest,
+			new ErrorResponseBody(errorMessage),
+			responseStatus,
+			Collections.emptyMap()
+		);
+	}
+
+	private void deleteUploadedFiles() {
+		if (currentUploadDir != null) {
+			try {
+				FileUtils.deleteDirectory(currentUploadDir.toFile());
+			} catch (IOException e) {
+				LOG.warn("Could not cleanup uploaded files.", e);
 			}
-		} else {
-			ctx.fireChannelRead(msg);
 		}
 	}
 
 	private void reset() {
+		// destroy() can fail because some data is stored multiple times in the decoder causing an IllegalReferenceCountException
+		// see https://github.com/netty/netty/issues/7814
+		currentHttpPostRequestDecoder.getBodyHttpDatas().clear();
 		currentHttpPostRequestDecoder.destroy();
 		currentHttpPostRequestDecoder = null;
 		currentHttpRequest = null;
+		currentUploadDir = null;
+		currentJsonPayload = null;
+	}
+
+	public static FileUploads getMultipartFileUploads(ChannelHandlerContext ctx) {
+		return Optional.ofNullable(ctx.channel().attr(UPLOADED_FILES).get())
+			.orElse(FileUploads.EMPTY);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
index f5a74c3..448711b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
@@ -70,7 +70,7 @@ public abstract class AbstractRestHandler<T extends RestfulGateway, R extends Re
 	}
 
 	@Override
-	protected void respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest<R, M> handlerRequest, T gateway) throws RestHandlerException {
+	protected void respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest<R, M> handlerRequest, T gateway) {
 		CompletableFuture<P> response;
 
 		try {
@@ -87,14 +87,7 @@ public abstract class AbstractRestHandler<T extends RestfulGateway, R extends Re
 				if (error instanceof RestHandlerException) {
 					final RestHandlerException rhe = (RestHandlerException) error;
 
-					log.error("Exception occurred in REST handler.", error);
-
-					HandlerUtils.sendErrorResponse(
-						ctx,
-						httpRequest,
-						new ErrorResponseBody(rhe.getMessage()),
-						rhe.getHttpResponseStatus(),
-						responseHeaders);
+					processRestHandlerException(ctx, httpRequest, rhe);
 				} else {
 					log.error("Implementation error: Unhandled exception.", error);
 					HandlerUtils.sendErrorResponse(
@@ -115,6 +108,17 @@ public abstract class AbstractRestHandler<T extends RestfulGateway, R extends Re
 		});
 	}
 
+	private void processRestHandlerException(ChannelHandlerContext ctx, HttpRequest httpRequest, RestHandlerException rhe) {
+		log.error("Exception occurred in REST handler.", rhe);
+
+		HandlerUtils.sendErrorResponse(
+			ctx,
+			httpRequest,
+			new ErrorResponseBody(rhe.getMessage()),
+			rhe.getHttpResponseStatus(),
+			responseHeaders);
+	}
+
 	/**
 	 * This method is called for every incoming request and returns a {@link CompletableFuture} containing a the response.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/FileUploads.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/FileUploads.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/FileUploads.java
new file mode 100644
index 0000000..31ac47bb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/FileUploads.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler;
+
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * A container for uploaded files.
+ *
+ * <p>Implementation note: The constructor also accepts directories to ensure that the upload directories are cleaned up.
+ * For convenience during testing it also accepts files directly.
+ */
+public final class FileUploads implements AutoCloseable {
+	@Nullable
+	private final Path uploadDirectory;
+
+	public static final FileUploads EMPTY = new FileUploads();
+
+	private FileUploads() {
+		this.uploadDirectory = null;
+	}
+
+	public FileUploads(@Nonnull Path uploadDirectory) {
+		Preconditions.checkNotNull(uploadDirectory, "UploadDirectory must not be null.");
+		Preconditions.checkArgument(Files.exists(uploadDirectory), "UploadDirectory does not exist.");
+		Preconditions.checkArgument(Files.isDirectory(uploadDirectory), "UploadDirectory is not a directory.");
+		Preconditions.checkArgument(uploadDirectory.isAbsolute(), "UploadDirectory is not absolute.");
+		this.uploadDirectory = uploadDirectory;
+	}
+
+	public Collection<Path> getUploadedFiles() throws IOException {
+		if (uploadDirectory == null) {
+			return Collections.emptyList();
+		}
+
+		FileAdderVisitor visitor = new FileAdderVisitor();
+		Files.walkFileTree(uploadDirectory, visitor);
+
+		return Collections.unmodifiableCollection(visitor.getContainedFiles());
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (uploadDirectory != null) {
+			FileUtils.deleteDirectory(uploadDirectory.toFile());
+		}
+	}
+
+	private static final class FileAdderVisitor extends SimpleFileVisitor<Path> {
+
+		private final Collection<Path> files = new ArrayList<>(4);
+
+		Collection<Path> getContainedFiles() {
+			return files;
+		}
+
+		FileAdderVisitor() {
+		}
+
+		@Override
+		public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+			FileVisitResult result = super.visitFile(file, attrs);
+			files.add(file);
+			return result;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java
index aacf0a2..7e93556 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/HandlerRequest.java
@@ -24,6 +24,10 @@ import org.apache.flink.runtime.rest.messages.MessageQueryParameter;
 import org.apache.flink.runtime.rest.messages.RequestBody;
 import org.apache.flink.util.Preconditions;
 
+import javax.annotation.Nonnull;
+
+import java.nio.file.Path;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -39,15 +43,21 @@ import java.util.StringJoiner;
 public class HandlerRequest<R extends RequestBody, M extends MessageParameters> {
 
 	private final R requestBody;
+	private final Collection<Path> uploadedFiles;
 	private final Map<Class<? extends MessagePathParameter<?>>, MessagePathParameter<?>> pathParameters = new HashMap<>(2);
 	private final Map<Class<? extends MessageQueryParameter<?>>, MessageQueryParameter<?>> queryParameters = new HashMap<>(2);
 
 	public HandlerRequest(R requestBody, M messageParameters) throws HandlerRequestException {
-		this(requestBody, messageParameters, Collections.emptyMap(), Collections.emptyMap());
+		this(requestBody, messageParameters, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyList());
 	}
 
 	public HandlerRequest(R requestBody, M messageParameters, Map<String, String> receivedPathParameters, Map<String, List<String>> receivedQueryParameters) throws HandlerRequestException {
+		this(requestBody, messageParameters, receivedPathParameters, receivedQueryParameters, Collections.emptyList());
+	}
+
+	public HandlerRequest(R requestBody, M messageParameters, Map<String, String> receivedPathParameters, Map<String, List<String>> receivedQueryParameters, Collection<Path> uploadedFiles) throws HandlerRequestException {
 		this.requestBody = Preconditions.checkNotNull(requestBody);
+		this.uploadedFiles = Collections.unmodifiableCollection(Preconditions.checkNotNull(uploadedFiles));
 		Preconditions.checkNotNull(messageParameters);
 		Preconditions.checkNotNull(receivedQueryParameters);
 		Preconditions.checkNotNull(receivedPathParameters);
@@ -129,4 +139,9 @@ public class HandlerRequest<R extends RequestBody, M extends MessageParameters>
 			return queryParameter.getValue();
 		}
 	}
+
+	@Nonnull
+	public Collection<Path> getUploadedFiles() {
+		return uploadedFiles;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/FileUpload.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/FileUpload.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/FileUpload.java
deleted file mode 100644
index fb09dce..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/FileUpload.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.rest.messages;
-
-import java.nio.file.Path;
-
-/**
- * Client uploading a file.
- */
-public class FileUpload implements RequestBody {
-
-	private final Path path;
-
-	public FileUpload(final Path path) {
-		this.path = path;
-	}
-
-	public Path getPath() {
-		return path;
-	}
-
-	@Override
-	public String toString() {
-		return "FileUpload{" +
-			"path=" + path +
-			'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/UntypedResponseMessageHeaders.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/UntypedResponseMessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/UntypedResponseMessageHeaders.java
index 1556b90..3de9626 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/UntypedResponseMessageHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/UntypedResponseMessageHeaders.java
@@ -40,4 +40,13 @@ public interface UntypedResponseMessageHeaders<R extends RequestBody, M extends
 	 * @return new message parameters object
 	 */
 	M getUnresolvedMessageParameters();
+
+	/**
+	 * Returns whether this header allows file uploads.
+	 *
+	 * @return whether this header allows file uploads
+	 */
+	default boolean acceptsFileUploads() {
+		return false;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/test/java/org/apache/flink/runtime/rest/AbstractHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/AbstractHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/AbstractHandlerTest.java
new file mode 100644
index 0000000..ffcc72a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/AbstractHandlerTest.java
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.rest.handler.FileUploads;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+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.HttpRequest;
+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.apache.flink.shaded.netty4.io.netty.util.Attribute;
+import org.apache.flink.shaded.netty4.io.netty.util.AttributeKey;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nonnull;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link AbstractHandler}.
+ */
+public class AbstractHandlerTest extends TestLogger {
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testFileCleanup() throws Exception {
+		final Path dir = temporaryFolder.newFolder().toPath();
+		final Path file = dir.resolve("file");
+		Files.createFile(file);
+
+		final String restAddress = "http://localhost:1234";
+		RestfulGateway mockRestfulGateway = TestingRestfulGateway.newBuilder()
+			.setRestAddress(restAddress)
+			.build();
+
+		final GatewayRetriever<RestfulGateway> mockGatewayRetriever = () ->
+			CompletableFuture.completedFuture(mockRestfulGateway);
+
+		TestHandler handler = new TestHandler(CompletableFuture.completedFuture(restAddress), mockGatewayRetriever);
+
+		HttpRequest request = new DefaultFullHttpRequest(
+			HttpVersion.HTTP_1_1,
+			HttpMethod.GET,
+			TestHandler.TestHeaders.INSTANCE.getTargetRestEndpointURL(),
+			Unpooled.wrappedBuffer(new byte[0]));
+		Routed routerRequest = new Routed("", false, request, request.getUri(), Collections.emptyMap(), Collections.emptyMap());
+
+		Attribute<FileUploads> attribute = new SimpleAttribute();
+		attribute.set(new FileUploads(dir));
+		Channel channel = mock(Channel.class);
+		when(channel.attr(any(AttributeKey.class))).thenReturn(attribute);
+
+		ChannelHandlerContext context = mock(ChannelHandlerContext.class);
+		when(context.channel()).thenReturn(channel);
+
+		handler.respondAsLeader(context, routerRequest, mockRestfulGateway);
+
+		Assert.assertFalse(Files.exists(file));
+	}
+
+	private static class SimpleAttribute implements Attribute<FileUploads> {
+
+		private static final AttributeKey<FileUploads> KEY = AttributeKey.valueOf("test");
+
+		private final AtomicReference<FileUploads> container = new AtomicReference<>();
+
+		@Override
+		public AttributeKey<FileUploads> key() {
+			return KEY;
+		}
+
+		@Override
+		public FileUploads get() {
+			return container.get();
+		}
+
+		@Override
+		public void set(FileUploads value) {
+			container.set(value);
+		}
+
+		@Override
+		public FileUploads getAndSet(FileUploads value) {
+			return container.getAndSet(value);
+		}
+
+		@Override
+		public FileUploads setIfAbsent(FileUploads value) {
+			if (container.compareAndSet(null, value)) {
+				return value;
+			} else {
+				return container.get();
+			}
+		}
+
+		@Override
+		public FileUploads getAndRemove() {
+			return container.getAndSet(null);
+		}
+
+		@Override
+		public boolean compareAndSet(FileUploads oldValue, FileUploads newValue) {
+			return container.compareAndSet(oldValue, newValue);
+		}
+
+		@Override
+		public void remove() {
+			set(null);
+		}
+	}
+
+	private static class TestHandler extends AbstractHandler<RestfulGateway, EmptyRequestBody, EmptyMessageParameters> {
+
+		protected TestHandler(@Nonnull CompletableFuture<String> localAddressFuture, @Nonnull GatewayRetriever<? extends RestfulGateway> leaderRetriever) {
+			super(localAddressFuture, leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), TestHeaders.INSTANCE);
+		}
+
+		@Override
+		protected void respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest<EmptyRequestBody, EmptyMessageParameters> handlerRequest, RestfulGateway gateway) throws RestHandlerException {
+
+		}
+
+		private enum TestHeaders implements UntypedResponseMessageHeaders<EmptyRequestBody, EmptyMessageParameters> {
+			INSTANCE;
+
+			@Override
+			public Class<EmptyRequestBody> getRequestClass() {
+				return EmptyRequestBody.class;
+			}
+
+			@Override
+			public EmptyMessageParameters getUnresolvedMessageParameters() {
+				return EmptyMessageParameters.getInstance();
+			}
+
+			@Override
+			public HttpMethodWrapper getHttpMethod() {
+				return HttpMethodWrapper.POST;
+			}
+
+			@Override
+			public String getTargetRestEndpointURL() {
+				return "/test";
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java
new file mode 100644
index 0000000..dd9a739
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java
@@ -0,0 +1,539 @@
+/*
+ * 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;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.WebOptions;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.rest.messages.RequestBody;
+import org.apache.flink.runtime.rest.util.RestMapperUtils;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import okhttp3.MediaType;
+import okhttp3.MultipartBody;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the {@link FileUploadHandler}. Ensures that multipart http messages containing files and/or json are properly
+ * handled.
+ */
+public class FileUploadHandlerTest extends TestLogger {
+
+	private static final ObjectMapper OBJECT_MAPPER = RestMapperUtils.getStrictObjectMapper();
+	private static final Random RANDOM = new Random();
+
+	@ClassRule
+	public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+	private static RestServerEndpoint serverEndpoint;
+	private static String serverAddress;
+
+	private static MultipartMixedHandler mixedHandler;
+	private static MultipartJsonHandler jsonHandler;
+	private static MultipartFileHandler fileHandler;
+	private static File file1;
+	private static File file2;
+
+	private static Path configuredUploadDir;
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		Configuration config = new Configuration();
+		config.setInteger(RestOptions.PORT, 0);
+		config.setString(RestOptions.ADDRESS, "localhost");
+		configuredUploadDir = TEMPORARY_FOLDER.newFolder().toPath();
+		config.setString(WebOptions.UPLOAD_DIR, configuredUploadDir.toString());
+
+		RestServerEndpointConfiguration serverConfig = RestServerEndpointConfiguration.fromConfiguration(config);
+
+		final String restAddress = "http://localhost:1234";
+		RestfulGateway mockRestfulGateway = TestingRestfulGateway.newBuilder()
+			.setRestAddress(restAddress)
+			.build();
+
+		final GatewayRetriever<RestfulGateway> mockGatewayRetriever = () ->
+			CompletableFuture.completedFuture(mockRestfulGateway);
+
+		file1 = TEMPORARY_FOLDER.newFile();
+		Files.write(file1.toPath(), "hello".getBytes(ConfigConstants.DEFAULT_CHARSET));
+		file2 = TEMPORARY_FOLDER.newFile();
+		Files.write(file2.toPath(), "world".getBytes(ConfigConstants.DEFAULT_CHARSET));
+
+		mixedHandler = new MultipartMixedHandler(CompletableFuture.completedFuture(restAddress), mockGatewayRetriever);
+		jsonHandler = new MultipartJsonHandler(CompletableFuture.completedFuture(restAddress), mockGatewayRetriever);
+		fileHandler = new MultipartFileHandler(CompletableFuture.completedFuture(restAddress), mockGatewayRetriever);
+
+		final List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = Arrays.asList(
+			Tuple2.of(mixedHandler.getMessageHeaders(), mixedHandler),
+			Tuple2.of(jsonHandler.getMessageHeaders(), jsonHandler),
+			Tuple2.of(fileHandler.getMessageHeaders(), fileHandler));
+
+		serverEndpoint = new TestRestServerEndpoint(serverConfig, handlers);
+
+		serverEndpoint.start();
+		serverAddress = serverEndpoint.getRestBaseUrl();
+	}
+
+	@AfterClass
+	public static void teardown() throws Exception {
+		if (serverEndpoint != null) {
+			serverEndpoint.close();
+			serverEndpoint = null;
+		}
+	}
+
+	private static Request buildMalformedRequest(String headerUrl) {
+		MultipartBody.Builder builder = new MultipartBody.Builder();
+		builder = addFilePart(builder);
+		// this causes a failure in the FileUploadHandler since the request should only contain form-data
+		builder = builder.addPart(okhttp3.RequestBody.create(MediaType.parse("text/plain"), "crash"));
+		return finalizeRequest(builder, headerUrl);
+	}
+
+	private static Request buildMixedRequestWithUnknownAttribute(String headerUrl) throws IOException {
+		MultipartBody.Builder builder = new MultipartBody.Builder();
+		builder = addJsonPart(builder, RANDOM.nextInt(), "hello");
+		builder = addFilePart(builder);
+		return finalizeRequest(builder, headerUrl);
+	}
+
+	private static Request buildFileRequest(String headerUrl) {
+		MultipartBody.Builder builder = new MultipartBody.Builder();
+		builder = addFilePart(builder);
+		return finalizeRequest(builder, headerUrl);
+	}
+
+	private static Request buildJsonRequest(String headerUrl, int index) throws IOException {
+		MultipartBody.Builder builder = new MultipartBody.Builder();
+		builder = addJsonPart(builder, index, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
+		return finalizeRequest(builder, headerUrl);
+	}
+
+	private static Request buildMixedRequest(String headerUrl, int index) throws IOException {
+		MultipartBody.Builder builder = new MultipartBody.Builder();
+		builder = addJsonPart(builder, index, FileUploadHandler.HTTP_ATTRIBUTE_REQUEST);
+		builder = addFilePart(builder);
+		return finalizeRequest(builder, headerUrl);
+	}
+
+	private static Request finalizeRequest(MultipartBody.Builder builder, String headerUrl) {
+		MultipartBody multipartBody = builder
+			.setType(MultipartBody.FORM)
+			.build();
+
+		return new Request.Builder()
+			.url(serverAddress + headerUrl)
+			.post(multipartBody)
+			.build();
+	}
+
+	private static MultipartBody.Builder addFilePart(MultipartBody.Builder builder) {
+		okhttp3.RequestBody filePayload1 = okhttp3.RequestBody.create(MediaType.parse("application/octet-stream"), file1);
+		okhttp3.RequestBody filePayload2 = okhttp3.RequestBody.create(MediaType.parse("application/octet-stream"), file2);
+
+		return builder.addFormDataPart("file1", file1.getName(), filePayload1)
+			.addFormDataPart("file2", file2.getName(), filePayload2);
+	}
+
+	private static MultipartBody.Builder addJsonPart(MultipartBody.Builder builder, int index, String attribute) throws IOException {
+		TestRequestBody jsonRequestBody = new TestRequestBody(index);
+
+		StringWriter sw = new StringWriter();
+		OBJECT_MAPPER.writeValue(sw, jsonRequestBody);
+
+		String jsonPayload = sw.toString();
+
+		return builder.addFormDataPart(attribute, jsonPayload);
+	}
+
+	@Test
+	public void testMixedMultipart() throws Exception {
+		OkHttpClient client = new OkHttpClient();
+
+		Request jsonRequest = buildJsonRequest(mixedHandler.getMessageHeaders().getTargetRestEndpointURL(), RANDOM.nextInt());
+		try (Response response = client.newCall(jsonRequest).execute()) {
+			// explicitly rejected by the test handler implementation
+			assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.code());
+		}
+
+		Request fileRequest = buildFileRequest(mixedHandler.getMessageHeaders().getTargetRestEndpointURL());
+		try (Response response = client.newCall(fileRequest).execute()) {
+			// expected JSON payload is missing
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+
+		int mixedId = RANDOM.nextInt();
+		Request mixedRequest = buildMixedRequest(mixedHandler.getMessageHeaders().getTargetRestEndpointURL(), mixedId);
+		try (Response response = client.newCall(mixedRequest).execute()) {
+			assertEquals(mixedHandler.getMessageHeaders().getResponseStatusCode().code(), response.code());
+			assertEquals(mixedId, mixedHandler.lastReceivedRequest.index);
+		}
+	}
+
+	@Test
+	public void testJsonMultipart() throws Exception {
+		OkHttpClient client = new OkHttpClient();
+
+		int jsonId = RANDOM.nextInt();
+		Request jsonRequest = buildJsonRequest(jsonHandler.getMessageHeaders().getTargetRestEndpointURL(), jsonId);
+		try (Response response = client.newCall(jsonRequest).execute()) {
+			assertEquals(jsonHandler.getMessageHeaders().getResponseStatusCode().code(), response.code());
+			assertEquals(jsonId, jsonHandler.lastReceivedRequest.index);
+		}
+
+		Request fileRequest = buildFileRequest(jsonHandler.getMessageHeaders().getTargetRestEndpointURL());
+		try (Response response = client.newCall(fileRequest).execute()) {
+			// either because JSON payload is missing or FileUploads are outright forbidden
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+
+		Request mixedRequest = buildMixedRequest(jsonHandler.getMessageHeaders().getTargetRestEndpointURL(), RANDOM.nextInt());
+		try (Response response = client.newCall(mixedRequest).execute()) {
+			// FileUploads are outright forbidden
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+	}
+
+	@Test
+	public void testFileMultipart() throws Exception {
+		OkHttpClient client = new OkHttpClient();
+
+		Request jsonRequest = buildJsonRequest(fileHandler.getMessageHeaders().getTargetRestEndpointURL(), RANDOM.nextInt());
+		try (Response response = client.newCall(jsonRequest).execute()) {
+			// JSON payload did not match expected format
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+
+		Request fileRequest = buildFileRequest(fileHandler.getMessageHeaders().getTargetRestEndpointURL());
+		try (Response response = client.newCall(fileRequest).execute()) {
+			assertEquals(fileHandler.getMessageHeaders().getResponseStatusCode().code(), response.code());
+		}
+
+		Request mixedRequest = buildMixedRequest(fileHandler.getMessageHeaders().getTargetRestEndpointURL(), RANDOM.nextInt());
+		try (Response response = client.newCall(mixedRequest).execute()) {
+			// JSON payload did not match expected format
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+	}
+
+	@Test
+	public void testUploadCleanupOnUnknownAttribute() throws IOException {
+		OkHttpClient client = new OkHttpClient();
+
+		Request request = buildMixedRequestWithUnknownAttribute(mixedHandler.getMessageHeaders().getTargetRestEndpointURL());
+		try (Response response = client.newCall(request).execute()) {
+			assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code());
+		}
+		assertUploadDirectoryIsEmpty();
+	}
+
+	/**
+	 * Crashes the handler be submitting a malformed multipart request and tests that the upload directory is cleaned up.
+	 */
+	@Test
+	public void testUploadCleanupOnFailure() throws IOException {
+		OkHttpClient client = new OkHttpClient();
+
+		Request request = buildMalformedRequest(mixedHandler.getMessageHeaders().getTargetRestEndpointURL());
+		try (Response response = client.newCall(request).execute()) {
+			// decoding errors aren't handled separately by the FileUploadHandler
+			assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.code());
+		}
+		assertUploadDirectoryIsEmpty();
+	}
+
+	private static void assertUploadDirectoryIsEmpty() throws IOException {
+		Preconditions.checkArgument(
+			1 == Files.list(configuredUploadDir).count(),
+			"Directory structure in rest upload directory has changed. Test must be adjusted");
+		Optional<Path> actualUploadDir = Files.list(configuredUploadDir).findAny();
+		Preconditions.checkArgument(
+			actualUploadDir.isPresent(),
+			"Expected upload directory does not exist.");
+		assertEquals("Not all files were cleaned up.", 0, Files.list(actualUploadDir.get()).count());
+	}
+
+	private static class MultipartMixedHandler extends AbstractRestHandler<RestfulGateway, TestRequestBody, EmptyResponseBody, EmptyMessageParameters> {
+		volatile TestRequestBody lastReceivedRequest = null;
+
+		MultipartMixedHandler(CompletableFuture<String> localRestAddress, GatewayRetriever<RestfulGateway> leaderRetriever) {
+			super(localRestAddress, leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), MultipartMixedHeaders.INSTANCE);
+		}
+
+		@Override
+		protected CompletableFuture<EmptyResponseBody> handleRequest(@Nonnull HandlerRequest<TestRequestBody, EmptyMessageParameters> request, @Nonnull RestfulGateway gateway) throws RestHandlerException {
+			MultipartFileHandler.verifyFileUpload(request.getUploadedFiles());
+			this.lastReceivedRequest = request.getRequestBody();
+			return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
+		}
+
+		private static final class MultipartMixedHeaders implements MessageHeaders<TestRequestBody, EmptyResponseBody, EmptyMessageParameters> {
+			private static final MultipartMixedHeaders INSTANCE = new MultipartMixedHeaders();
+
+			private MultipartMixedHeaders() {
+			}
+
+			@Override
+			public Class<TestRequestBody> getRequestClass() {
+				return TestRequestBody.class;
+			}
+
+			@Override
+			public Class<EmptyResponseBody> getResponseClass() {
+				return EmptyResponseBody.class;
+			}
+
+			@Override
+			public HttpResponseStatus getResponseStatusCode() {
+				return HttpResponseStatus.OK;
+			}
+
+			@Override
+			public String getDescription() {
+				return "";
+			}
+
+			@Override
+			public EmptyMessageParameters getUnresolvedMessageParameters() {
+				return EmptyMessageParameters.getInstance();
+			}
+
+			@Override
+			public HttpMethodWrapper getHttpMethod() {
+				return HttpMethodWrapper.POST;
+			}
+
+			@Override
+			public String getTargetRestEndpointURL() {
+				return "/test/upload/mixed";
+			}
+
+			@Override
+			public boolean acceptsFileUploads() {
+				return true;
+			}
+		}
+	}
+
+	private static class MultipartJsonHandler extends AbstractRestHandler<RestfulGateway, TestRequestBody, EmptyResponseBody, EmptyMessageParameters> {
+		volatile TestRequestBody lastReceivedRequest = null;
+
+		MultipartJsonHandler(CompletableFuture<String> localRestAddress, GatewayRetriever<RestfulGateway> leaderRetriever) {
+			super(localRestAddress, leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), MultipartJsonHeaders.INSTANCE);
+		}
+
+		@Override
+		protected CompletableFuture<EmptyResponseBody> handleRequest(@Nonnull HandlerRequest<TestRequestBody, EmptyMessageParameters> request, @Nonnull RestfulGateway gateway) throws RestHandlerException {
+			Collection<Path> uploadedFiles = request.getUploadedFiles();
+			if (!uploadedFiles.isEmpty()) {
+				throw new RestHandlerException("This handler should not have received file uploads.", HttpResponseStatus.INTERNAL_SERVER_ERROR);
+			}
+			this.lastReceivedRequest = request.getRequestBody();
+			return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
+		}
+
+		private static final class MultipartJsonHeaders extends TestHeadersBase<TestRequestBody> {
+			private static final MultipartJsonHeaders INSTANCE = new MultipartJsonHeaders();
+
+			private MultipartJsonHeaders() {
+			}
+
+			@Override
+			public Class<TestRequestBody> getRequestClass() {
+				return TestRequestBody.class;
+			}
+
+			@Override
+			public String getTargetRestEndpointURL() {
+				return "/test/upload/json";
+			}
+
+			@Override
+			public boolean acceptsFileUploads() {
+				return false;
+			}
+		}
+	}
+
+	private static class MultipartFileHandler extends AbstractRestHandler<RestfulGateway, EmptyRequestBody, EmptyResponseBody, EmptyMessageParameters> {
+
+		MultipartFileHandler(CompletableFuture<String> localRestAddress, GatewayRetriever<RestfulGateway> leaderRetriever) {
+			super(localRestAddress, leaderRetriever, RpcUtils.INF_TIMEOUT, Collections.emptyMap(), MultipartFileHeaders.INSTANCE);
+		}
+
+		@Override
+		protected CompletableFuture<EmptyResponseBody> handleRequest(@Nonnull HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, @Nonnull RestfulGateway gateway) throws RestHandlerException {
+			verifyFileUpload(request.getUploadedFiles());
+			return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
+		}
+
+		static void verifyFileUpload(Collection<Path> uploadedFiles) throws RestHandlerException {
+			try {
+				assertEquals(2, uploadedFiles.size());
+
+				for (Path uploadedFile : uploadedFiles) {
+					File matchingFile;
+					if (uploadedFile.getFileName().toString().equals(file1.getName())) {
+						matchingFile = file1;
+					} else if (uploadedFile.getFileName().toString().equals(file2.getName())) {
+						matchingFile = file2;
+					} else {
+						throw new RestHandlerException("Received file with unknown name " + uploadedFile.getFileName() + '.', HttpResponseStatus.INTERNAL_SERVER_ERROR);
+					}
+
+					byte[] originalContent = Files.readAllBytes(matchingFile.toPath());
+					byte[] receivedContent = Files.readAllBytes(uploadedFile);
+					assertArrayEquals(originalContent, receivedContent);
+				}
+			} catch (Exception e) {
+				// return 505 to differentiate from common BAD_REQUEST responses in this test
+				throw new RestHandlerException("Test verification failed.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
+			}
+		}
+
+		private static final class MultipartFileHeaders extends TestHeadersBase<EmptyRequestBody> {
+			private static final MultipartFileHeaders INSTANCE = new MultipartFileHeaders();
+
+			private MultipartFileHeaders() {
+			}
+
+			@Override
+			public Class<EmptyRequestBody> getRequestClass() {
+				return EmptyRequestBody.class;
+			}
+
+			@Override
+			public String getTargetRestEndpointURL() {
+				return "/test/upload/file";
+			}
+
+			@Override
+			public boolean acceptsFileUploads() {
+				return true;
+			}
+		}
+	}
+
+	private abstract static class TestHeadersBase<R extends RequestBody> implements MessageHeaders<R, EmptyResponseBody, EmptyMessageParameters> {
+
+		@Override
+		public Class<EmptyResponseBody> getResponseClass() {
+			return EmptyResponseBody.class;
+		}
+
+		@Override
+		public HttpResponseStatus getResponseStatusCode() {
+			return HttpResponseStatus.OK;
+		}
+
+		@Override
+		public String getDescription() {
+			return "";
+		}
+
+		@Override
+		public EmptyMessageParameters getUnresolvedMessageParameters() {
+			return EmptyMessageParameters.getInstance();
+		}
+
+		@Override
+		public HttpMethodWrapper getHttpMethod() {
+			return HttpMethodWrapper.POST;
+		}
+	}
+
+	private static final class TestRequestBody implements RequestBody {
+		private static final String FIELD_NAME_INDEX = "index";
+
+		@JsonProperty(FIELD_NAME_INDEX)
+		private final int index;
+
+		@JsonCreator
+		TestRequestBody(@JsonProperty(FIELD_NAME_INDEX) int index) {
+			this.index = index;
+		}
+	}
+
+	private static class TestRestServerEndpoint extends RestServerEndpoint {
+
+		private final List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers;
+
+		TestRestServerEndpoint(
+			RestServerEndpointConfiguration configuration,
+			List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers) throws IOException {
+			super(configuration);
+			this.handlers = Preconditions.checkNotNull(handlers);
+		}
+
+		@Override
+		protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+			return handlers;
+		}
+
+		@Override
+		protected void startInternal() {
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java
index dad3b4f..b9413ba 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java
@@ -34,8 +34,8 @@ import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandle
 import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification;
 import org.apache.flink.runtime.rest.messages.ConversionException;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
-import org.apache.flink.runtime.rest.messages.FileUpload;
 import org.apache.flink.runtime.rest.messages.MessageHeaders;
 import org.apache.flink.runtime.rest.messages.MessageParameters;
 import org.apache.flink.runtime.rest.messages.MessagePathParameter;
@@ -367,8 +367,8 @@ public class RestServerEndpointITCase extends TestLogger {
 		}
 
 		assertEquals(200, connection.getResponseCode());
-		final Path lastUploadedPath = testUploadHandler.getLastUploadedPath();
-		assertEquals(uploadedContent, new String(Files.readAllBytes(lastUploadedPath), StandardCharsets.UTF_8));
+		final byte[] lastUploadedFileContents = testUploadHandler.getLastUploadedFileContents();
+		assertEquals(uploadedContent, new String(lastUploadedFileContents, StandardCharsets.UTF_8));
 	}
 
 	/**
@@ -648,9 +648,9 @@ public class RestServerEndpointITCase extends TestLogger {
 		}
 	}
 
-	private static class TestUploadHandler extends AbstractRestHandler<RestfulGateway, FileUpload, EmptyResponseBody, EmptyMessageParameters> {
+	private static class TestUploadHandler extends AbstractRestHandler<RestfulGateway, EmptyRequestBody, EmptyResponseBody, EmptyMessageParameters> {
 
-		private volatile Path lastUploadedPath;
+		private volatile byte[] lastUploadedFileContents;
 
 		private TestUploadHandler(
 			final CompletableFuture<String> localRestAddress,
@@ -660,17 +660,26 @@ public class RestServerEndpointITCase extends TestLogger {
 		}
 
 		@Override
-		protected CompletableFuture<EmptyResponseBody> handleRequest(@Nonnull final HandlerRequest<FileUpload, EmptyMessageParameters> request, @Nonnull final RestfulGateway gateway) throws RestHandlerException {
-			lastUploadedPath = request.getRequestBody().getPath();
+		protected CompletableFuture<EmptyResponseBody> handleRequest(@Nonnull final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, @Nonnull final RestfulGateway gateway) throws RestHandlerException {
+			Collection<Path> uploadedFiles = request.getUploadedFiles();
+			if (uploadedFiles.size() != 1) {
+				throw new RestHandlerException("Expected 1 file, received " + uploadedFiles.size() + '.', HttpResponseStatus.BAD_REQUEST);
+			}
+
+			try {
+				lastUploadedFileContents = Files.readAllBytes(uploadedFiles.iterator().next());
+			} catch (IOException e) {
+				throw new RestHandlerException("Could not read contents of uploaded file.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
+			}
 			return CompletableFuture.completedFuture(EmptyResponseBody.getInstance());
 		}
 
-		public Path getLastUploadedPath() {
-			return lastUploadedPath;
+		public byte[] getLastUploadedFileContents() {
+			return lastUploadedFileContents;
 		}
 	}
 
-	private enum TestUploadHeaders implements MessageHeaders<FileUpload, EmptyResponseBody, EmptyMessageParameters> {
+	private enum TestUploadHeaders implements MessageHeaders<EmptyRequestBody, EmptyResponseBody, EmptyMessageParameters> {
 		INSTANCE;
 
 		@Override
@@ -684,8 +693,8 @@ public class RestServerEndpointITCase extends TestLogger {
 		}
 
 		@Override
-		public Class<FileUpload> getRequestClass() {
-			return FileUpload.class;
+		public Class<EmptyRequestBody> getRequestClass() {
+			return EmptyRequestBody.class;
 		}
 
 		@Override
@@ -707,5 +716,10 @@ public class RestServerEndpointITCase extends TestLogger {
 		public String getDescription() {
 			return "";
 		}
+
+		@Override
+		public boolean acceptsFileUploads() {
+			return true;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e0265062/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/FileUploadsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/FileUploadsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/FileUploadsTest.java
new file mode 100644
index 0000000..fb7faa3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/FileUploadsTest.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;
+
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+
+/**
+ * Tests for {@link FileUploads}.
+ */
+public class FileUploadsTest extends TestLogger {
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testRelativePathRejection() throws IOException {
+		Path relative = Paths.get("root");
+		try {
+			new FileUploads(relative);
+			Assert.fail();
+		} catch (IllegalArgumentException iae) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testDirectoryScan() throws IOException {
+		Path rootDir = Paths.get("root");
+		Path rootFile = rootDir.resolve("rootFile");
+		Path subDir = rootDir.resolve("sub");
+		Path subFile = subDir.resolve("subFile");
+
+		Path tmp = temporaryFolder.getRoot().toPath();
+		Files.createDirectory(tmp.resolve(rootDir));
+		Files.createDirectory(tmp.resolve(subDir));
+		Files.createFile(tmp.resolve(rootFile));
+		Files.createFile(tmp.resolve(subFile));
+
+		try (FileUploads fileUploads = new FileUploads(tmp.resolve(rootDir))) {
+			Collection<Path> detectedFiles = fileUploads.getUploadedFiles();
+
+			Assert.assertEquals(2, detectedFiles.size());
+			Assert.assertTrue(detectedFiles.contains(tmp.resolve(rootFile)));
+			Assert.assertTrue(detectedFiles.contains(tmp.resolve(subFile)));
+		}
+	}
+
+	@Test
+	public void testEmptyDirectory() throws IOException {
+		Path rootDir = Paths.get("root");
+
+		Path tmp = temporaryFolder.getRoot().toPath();
+		Files.createDirectory(tmp.resolve(rootDir));
+
+		try (FileUploads fileUploads = new FileUploads(tmp.resolve(rootDir))) {
+			Collection<Path> detectedFiles = fileUploads.getUploadedFiles();
+			Assert.assertEquals(0, detectedFiles.size());
+		}
+	}
+
+	@Test
+	public void testCleanup() throws IOException {
+		Path rootDir = Paths.get("root");
+		Path rootFile = rootDir.resolve("rootFile");
+		Path subDir = rootDir.resolve("sub");
+		Path subFile = subDir.resolve("subFile");
+
+		Path tmp = temporaryFolder.getRoot().toPath();
+		Files.createDirectory(tmp.resolve(rootDir));
+		Files.createDirectory(tmp.resolve(subDir));
+		Files.createFile(tmp.resolve(rootFile));
+		Files.createFile(tmp.resolve(subFile));
+
+		try (FileUploads fileUploads = new FileUploads(tmp.resolve(rootDir))) {
+			Assert.assertTrue(Files.exists(tmp.resolve(rootDir)));
+			Assert.assertTrue(Files.exists(tmp.resolve(subDir)));
+			Assert.assertTrue(Files.exists(tmp.resolve(rootFile)));
+			Assert.assertTrue(Files.exists(tmp.resolve(subFile)));
+		}
+		Assert.assertFalse(Files.exists(tmp.resolve(rootDir)));
+		Assert.assertFalse(Files.exists(tmp.resolve(subDir)));
+		Assert.assertFalse(Files.exists(tmp.resolve(rootFile)));
+		Assert.assertFalse(Files.exists(tmp.resolve(subFile)));
+	}
+}


[3/5] flink git commit: [FLINK-9467][metrics] Rebuild UI

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4b2332a4/flink-runtime-web/web-dashboard/web/js/hs/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/hs/index.js b/flink-runtime-web/web-dashboard/web/js/hs/index.js
index 2e17c18..4c3a138 100644
--- a/flink-runtime-web/web-dashboard/web/js/hs/index.js
+++ b/flink-runtime-web/web-dashboard/web/js/hs/index.js
@@ -1,2 +1,2 @@
-angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,i){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsC
 ontroller"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanagers.html",controller:"JobPlanTaskManagersController"}}}).
 state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.history.html",controll
 er:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-job.exceptions",{url:"/exceptions",views:{details:{templ
 ateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}),t.otherwise("/completed-jobs")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["Jobs
 Service",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d 
 "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e<i?(e/i).toFixed(2)+" "+r[n]:e<1e3*i?(e/i).toPrecision(3)+" "+r[n]:t(e,n+1)},"undefined"==typeof e||null===e?"":e<1e3?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filter("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}).filter("humanizeWatermark",["watermarksConfig",function(e){return function(t){return isNaN(t)||t<=e.noWatermark?"No Watermark":t}}]).filter("increment",function(){return function(e){return parseI
 nt(e)+1}}).filter("humanizeChartNumeric",["humanizeBytesFilter","humanizeDurationFilter",function(e,t){return function(r,n){var i;return i="",null!==r&&(i=/bytes/i.test(n.id)&&/persecond/i.test(n.id)?e(r)+" / s":/bytes/i.test(n.id)?e(r):/persecond/i.test(n.id)?r+" / s":/time/i.test(n.id)||/latency/i.test(n.id)?t(r,!0):r),i}}]).filter("humanizeChartNumericTitle",["humanizeDurationFilter",function(e){return function(t,r){var n;return n="",null!==t&&(n=/bytes/i.test(r.id)&&/persecond/i.test(r.id)?t+" Bytes / s":/bytes/i.test(r.id)?t+" Bytes":/persecond/i.test(r.id)?t+" / s":/time/i.test(r.id)||/latency/i.test(r.id)?e(t,!1):t),n}}]).filter("searchMetrics",function(){return function(e,t){var r,n;return n=new RegExp(t,"gi"),function(){var t,i,o;for(o=[],t=0,i=e.length;t<i;t++)r=e[t],r.id.match(n)&&o.push(r);return o}()}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer
 +"config").success(function(e,t,r,i){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){return t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadData=function(){return t.loadLogs().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;re
 turn n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,i){return i=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var i;return i=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadStdout=function(){var i;return i=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver
 ()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval","$q","watermarksConfig",function(e,t,r,n,i,o,s,a,l,u){var c,d;return e.jobid=r.jobid,e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats={},d=a(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},s["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats=null,a.cancel(d)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling
 ..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})},n.loadJob(r.jobid).then(function(t){return e.job=t,e.vertices=t.vertices,e.plan=t.plan,i.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,o,s;return o=function(t){return function(t){var r,n,o,s;return r=l.defer(),o=e.job.jid,s=function(){var e,r,i;for(i=[],n=e=0,r=t.parallelism-1;0<=r?e<=r:e>=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentLowWatermark",""),l[s]=a,(isNaN(i)||a<i)&&(i=a);return n=!isNaN(i)&&i>u.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,
 t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(
 ){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return 
 n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorS
 tats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){re
 turn e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),r<n?-1:r>n?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return
  i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).pref
 ix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace("&gt;",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{
 jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph'><g /></svg> <svg class='tmp' width='1' height='1'><g /></svg> <div class='btn-group zoom-buttons'> <a class='btn btn-default zoom-in' ng-click='zoomIn()'><i class='fa fa-plus' /></a> <a class='btn btn-default zoom-out' ng-click='zoomOut()'><i class='fa fa-minus' /></a> </div>",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3
 .select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workse
 t"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="<div href='#/jobs/"+g+"/vertex/"+e.id+"' class='node-label "+m(e,t)+"'>",i+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?i+="":(o=e.description,o=J(o),i+="<h4 class='step-name'>"+o+"</h4>"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(i+="<h5>Parallelism: "+e.parallelism+"</h5>"),void 0!==e.lowWatermark&&(i+="<h5>Low Watermark: "+e.lowWatermark+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(i+="<h5>Operation: "+J(e.operator_strategy)+"</h5>")),i+="</div>"},f=function(e,t,r){var n,i;return i="svg-"+e,n="<svg class='"+i+"' width="+t+" height="+r+"><g /></svg>"},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e.len
 gth>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal
 "})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s<u;s++)if(r=v[s],p=0,f=0,r.step_function&&(b=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),M[r.id]=b,k(b,r),h=new dagreD3.render,c.select("g").call(h,b),p=b.graph().width,f=b.graph().height,angular.element(w).empty()),a(e,t,r,o,p,f),n.push(r.id),null!=r.inputs)for(g=r.inputs,l=0,d=g.length;l<d;l++)m=g[l],i(e,t,r,n,m);return e},y=function(e,t){var r,n,i;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)for(i in r.step_function)if(r.step_function[i].id===t)return r.step_function[i]}},$=function(e,t){var r,n,i,o;if(!_.isEmpty(t))for(o=e.nodes,r=0,n=o.length;r<n;r++)i=o[r],t[i.id]&&!isNaN(t[i.id].lowWatermark)&&(i.lowWatermark=t[i.id].lowWatermark);return e},b=0,v=0,d=function(){var t,r,n,i,o,s;if(e.plan){p=new dagreD3.graphlib.Graph({multigraph:!0
 ,compound:!0}).setGraph({nodesep:70,edgesep:0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),k(p,$(e.plan,e.watermarks)),u.selectAll("*").remove(),u.attr("transform","scale(1)"),n=new dagreD3.render,u.call(n,p);for(t in M)i=M[t],l.select("svg.svg-"+t+" g").call(n,i);return r=.5,o=Math.floor((angular.element(S).width()-p.graph().width*r)/2),s=Math.floor((angular.element(S).height()-p.graph().height*r)/2),0!==v&&0!==b?(C.scale(v).translate(b),u.attr("transform","translate("+b+") scale("+v+")")):(C.scale(r).translate([o,s]),u.attr("transform","translate("+o+", "+s+") scale("+C.scale()+")")),C.on("zoom",function(){var e;return e=d3.event,v=e.scale,b=e.translate,u.attr("transform","translate("+b+") scale("+v+")")}),C(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})}},e.$watch(r.plan,function(e){if(e)return d()}),e.$watch(r.watermarks,function(t){if(t&&e.plan)return d()})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log","amM
 oment","$q","$timeout",function(e,t,r,n,i,o){var s,a,l,u,c,d;return s=null,a=null,l={},c={running:[],finished:[],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLowerCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){if(!(e["end-time"]>-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){ret
 urn e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=
 angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n<i;n++){if(o=t[n],o.id===e)return o;if(o.step_function&&(s=r(e,o.step_function)),s)return s}return null},t=i.defer(),l.job.promise.then(function(n){return function(i){var o;return o=r(e,s.plan.nodes),o.vertex=n.seekVertex(e),t.resolve(o)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,i;for(n=s.vertices,t=0,r=n.length;t<r;t++)if(i=n[t],i.id===e)return i;return null},this.getVertex=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(o){var a;return a=i.seekVertex(r),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return a.subtasks=e.subtasks,n.resolve(a)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r).success(function(e){var
  t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return console.log(s.jid),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/accumulators").success(function(i){var o;return o=i["user-accumulators"],e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:o,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.resolv
 e(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints").success(function(e,t,n,i){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var o;return o=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?o.resolve(null):o.resolve(e)})}}(this)),o.promise},this.getOperatorBackPressure=function(r){var n;return n=i.defer(),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/backp
 ressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPressureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"danger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/exceptions").success(function(e){return s.exceptions=e,r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel")},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.si
 ze != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {active: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div> <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg ng-if="metric.view == \'chart\'"/> <div ng-if="metric.view != \'chart\'"> <div class="metric-numeric" title="{{value | humanizeChartNumericTitle:metric}}">{{value | humanizeChartNumeric:metric}}</div> </div> </div> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.view == \'chart\'}]" ng-click="setView(\'chart\')">Chart</button> <button type="button" ng-class="[btnClasses, {active: metric.view != \'chart\'}]" ng-click="setView(\'numeric\')">Numeric</button> </div> </div>',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:func
 tion(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()
-}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t<Math.pow(10,n+i)?r=!0:n+=i;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.v
 alue=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-in
 terval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.f
 orEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r
 ].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s<n))return n-=1}}(this)),this.metrics[e][t].splice(n,0,r),this.saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){return _.isString(e)?{id:e,size:"small",view:"chart"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(i,o){var s;return n.setupLSFor(i,o),s=t.defer(),e.get(r.jobServer+"jobs/"+i+"/vertices/"+o+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var s;if(s=n.metrics[i][o].indexOf(e.id),s===-1&&(s=_.findIndex(n.metrics[i][o],{id:e.id})),s===-1)return t.push(e)}),s.resolve(t)}),s.promise}}(this),this.getAllAvailableMetrics=function(n){return func
 tion(n,i){var o;return o=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"/metrics").success(function(e){return o.resolve(e)}),o.promise}}(this),this.getMetrics=function(n,i,o){var s,a;return s=t.defer(),a=o.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"/metrics?get="+a).success(function(e){return function(t){var r,o;return o={},angular.forEach(t,function(e,t){return o[e.id]=parseInt(e.value)}),r={timestamp:Date.now(),values:o},e.saveValue(n,i,r),s.resolve(r)}}(this)),s.promise},this.setupLS(),this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,i){var o;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),o=n(function(){return t.loadOverv
 iew().then(function(t){return e.overview=t})},i["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(o)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadOverview=function(){var i;return i=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","$state","$location",function(e,t,r,n,i,o){var s;return e.yarn=o.absUrl().indexOf("/proxy/application_")!==-1,e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,e.noaccess=t.error,e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","submit-button":"Submit","action-time":0}},e.defaultStat
 e(),e.uploader={},e.loadList(),s=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(s)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spinner"),t.deleteJar(n).then(function(e){if(angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error)return alert(e.error)})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r,n;if("Show Plan"===e.state["plan-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry-class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["progra
 m-args"]&&(n["program-args"]=e.state["program-args"]),t.getPlan(e.state.selected,n).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})["catch"](function(t){return e.state["plan-button"]="Show Plan",e.error=t})},e.runJob=function(){var r,n;if("Submit"===e.state["submit-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submitting",e.state["plan-button"]="Show Plan",e.error=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry-class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["program-args"]&&(n["program-args"]=e.state["program-args"]),e.state.savepointPath&&(n.savepointPath=e.state.savepointPath),e.state.allowNonRestoredState&&(n.allowNonRestoredState=e.state.allowNonRestoredState),t.runJob(e.state.selected,n).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return i.go("singl
 e-job.plan.subtasks",{jobid:t.jobid})})["catch"](function(t){return e.state["submit-button"]="Submit",e.error=t})},e.nodeid=null,e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.$broadcast("reload")):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null)},e.clearFiles=function(){return e.uploader={}},e.uploadFiles=function(t){return e.uploader={},1===t.length?(e.uploader.file=t[0],e.uploader.upload=!0):e.uploader.error="Did ya forget to select a file?"},e.startUpload=function(){var t,r;return null!=e.uploader.file?(t=new FormData,t.append("jarfile",e.uploader.file),e.uploader.upload=!1,e.uploader.success="Initializing upload...",r=new XMLHttpRequest,r.upload.onprogress=function(t){return e.uploader.success=null,e.uploader.progress=parseInt(100*t.loaded/t.total)},r.upload.onerror=function(t){return e.uploader.progress=null,e.uploader.error="An error occurred while uploading your file"},r.upload.o
 nload=function(t){return e.uploader.progress=null,e.uploader.success="Saving..."},r.onreadystatechange=function(){var t;if(4===r.readyState)return t=JSON.parse(r.responseText),null!=t.error?(e.uploader.error=t.error,e.uploader.success=null):e.uploader.success="Uploaded!"},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelectClass",function(){return function(e,t){return e===t?"fa-check-square":"fa-square-o"}}),angular.module("flinkApp").service("JobSubmitService",["$http","flinkConfig","$q",function(e,t,r){return this.loadJarList=function(){var n;return n=r.defer(),e.get(t.jobServer+"jars/").success(function(e,t,r,i){return n.resolve(e)}),n.promise},this.deleteJar=function(n){var i;return i=r.defer(),e["delete"](t.jobServer+"jars/"+encodeURIComponent(n)).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.getPlan=function(n,i){var o;return o=r.defer(),e.get(t.jobServer+"jars/"+encodeURIComponen
 t(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(e)}).error(function(e){return o.reject(e)}),o.promise},this.runJob=function(n,i){var o;return o=r.defer(),e.post(t.jobServer+"jars/"+encodeURIComponent(n)+"/run",{},{params:i}).success(function(e,t,r,n){return o.resolve(e)}).error(function(e){return o.reject(e)}),o.promise},this}]),angular.module("flinkApp").controller("AllTaskManagersController",["$scope","TaskManagersService","$interval","flinkConfig",function(e,t,r,n){var i;return t.loadManagers().then(function(t){return e.managers=t}),i=r(function(){return t.loadManagers().then(function(t){return e.managers=t})},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(i)})}]).controller("SingleTaskManagerController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,i){var o;return e.metrics={},r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t}),o=n(function(){return r.loadMetrics(t.taskmanag
 erid).then(function(t){return e.metrics=t})},i["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(o)})}]).controller("SingleTaskManagerLogsController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,i){return e.log={},e.taskmanagerid=t.taskmanagerid,r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t}),e.reloadData=function(){return r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t})}}]).controller("SingleTaskManagerStdoutController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,i){return e.stdout={},e.taskmanagerid=t.taskmanagerid,r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t}),e.reloadData=function(){return r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t})}}]),angular.module("flinkApp").service("TaskManagersService",["$http","flinkConfig","$q",function(e,t,r){return this.loadManagers=function(){var n;return n=r.defer(),
 e.get(t.jobServer+"taskmanagers").success(function(e,t,r,i){return n.resolve(e.taskmanagers)}),n.promise},this}]).service("SingleTaskManagerService",["$http","flinkConfig","$q",function(e,t,r){return this.loadMetrics=function(n){var i;return i=r.defer(),e.get(t.jobServer+"taskmanagers/"+n).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.loadLogs=function(n){var i;return i=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/log").success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.loadStdout=function(n){var i;return i=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/stdout").success(function(e,t,r,n){return i.resolve(e)}),i.promise},this}]);
\ No newline at end of file
+angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,i){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsC
 ontroller"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanagers.html",controller:"JobPlanTaskManagersController"}}}).
 state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.history.html",controll
 er:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-job.exceptions",{url:"/exceptions",views:{details:{templ
 ateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}),t.otherwise("/completed-jobs")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["Jobs
 Service",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d 
 "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e<i?(e/i).toFixed(2)+" "+r[n]:e<1e3*i?(e/i).toPrecision(3)+" "+r[n]:t(e,n+1)},"undefined"==typeof e||null===e?"":e<1e3?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filter("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}).filter("humanizeWatermark",["watermarksConfig",function(e){return function(t){return isNaN(t)||t<=e.noWatermark?"No Watermark":t}}]).filter("increment",function(){return function(e){return parseI
 nt(e)+1}}).filter("humanizeChartNumeric",["humanizeBytesFilter","humanizeDurationFilter",function(e,t){return function(r,n){var i;return i="",null!==r&&(i=/bytes/i.test(n.id)&&/persecond/i.test(n.id)?e(r)+" / s":/bytes/i.test(n.id)?e(r):/persecond/i.test(n.id)?r+" / s":/time/i.test(n.id)||/latency/i.test(n.id)?t(r,!0):r),i}}]).filter("humanizeChartNumericTitle",["humanizeDurationFilter",function(e){return function(t,r){var n;return n="",null!==t&&(n=/bytes/i.test(r.id)&&/persecond/i.test(r.id)?t+" Bytes / s":/bytes/i.test(r.id)?t+" Bytes":/persecond/i.test(r.id)?t+" / s":/time/i.test(r.id)||/latency/i.test(r.id)?e(t,!1):t),n}}]).filter("searchMetrics",function(){return function(e,t){var r,n;return n=new RegExp(t,"gi"),function(){var t,i,o;for(o=[],t=0,i=e.length;t<i;t++)r=e[t],r.id.match(n)&&o.push(r);return o}()}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer
 +"config").success(function(e,t,r,i){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){return t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadData=function(){return t.loadLogs().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;re
 turn n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,i){return i=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var i;return i=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadStdout=function(){var i;return i=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver
 ()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval","$q","watermarksConfig",function(e,t,r,n,i,o,s,a,l,u){var c,d;return e.jobid=r.jobid,e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats={},d=a(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},s["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats=null,a.cancel(d)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling
 ..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})},n.loadJob(r.jobid).then(function(t){return e.job=t,e.vertices=t.vertices,e.plan=t.plan,i.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,o,s;return o=function(t){return function(t){var r,n,o,s;return r=l.defer(),o=e.job.jid,s=function(){var e,r,i;for(i=[],n=e=0,r=t.parallelism-1;0<=r?e<=r:e>=r;n=0<=r?++e:--e)i.push(n+".currentInputWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentInputWatermark",""),l[s]=a,(isNaN(i)||a<i)&&(i=a);return n=!isNaN(i)&&i>u.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i
 ]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=funct
 ion(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){ret
 urn n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOpera
 torStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t
 ){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),r<n?-1:r>n?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)re
 turn i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).
 prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace("&gt;",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.verte
 x",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph'><g /></svg> <svg class='tmp' width='1' height='1'><g /></svg> <div class='btn-group zoom-buttons'> <a class='btn btn-default zoom-in' ng-click='zoomIn()'><i class='fa fa-plus' /></a> <a class='btn btn-default zoom-out' ng-click='zoomOut()'><i class='fa fa-minus' /></a> </div>",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],
 l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"wo
 rkset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="<div href='#/jobs/"+g+"/vertex/"+e.id+"' class='node-label "+m(e,t)+"'>",i+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?i+="":(o=e.description,o=J(o),i+="<h4 class='step-name'>"+o+"</h4>"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(i+="<h5>Parallelism: "+e.parallelism+"</h5>"),void 0!==e.lowWatermark&&(i+="<h5>Low Watermark: "+e.lowWatermark+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(i+="<h5>Operation: "+J(e.operator_strategy)+"</h5>")),i+="</div>"},f=function(e,t,r){var n,i;return i="svg-"+e,n="<svg class='"+i+"' width="+t+" height="+r+"><g /></svg>"},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e
 .length>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"no
 rmal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s<u;s++)if(r=v[s],p=0,f=0,r.step_function&&(b=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),M[r.id]=b,k(b,r),h=new dagreD3.render,c.select("g").call(h,b),p=b.graph().width,f=b.graph().height,angular.element(w).empty()),a(e,t,r,o,p,f),n.push(r.id),null!=r.inputs)for(g=r.inputs,l=0,d=g.length;l<d;l++)m=g[l],i(e,t,r,n,m);return e},y=function(e,t){var r,n,i;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)for(i in r.step_function)if(r.step_function[i].id===t)return r.step_function[i]}},$=function(e,t){var r,n,i,o;if(!_.isEmpty(t))for(o=e.nodes,r=0,n=o.length;r<n;r++)i=o[r],t[i.id]&&!isNaN(t[i.id].lowWatermark)&&(i.lowWatermark=t[i.id].lowWatermark);return e},b=0,v=0,d=function(){var t,r,n,i,o,s;if(e.plan){p=new dagreD3.graphlib.Graph({multigrap
 h:!0,compound:!0}).setGraph({nodesep:70,edgesep:0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),k(p,$(e.plan,e.watermarks)),u.selectAll("*").remove(),u.attr("transform","scale(1)"),n=new dagreD3.render,u.call(n,p);for(t in M)i=M[t],l.select("svg.svg-"+t+" g").call(n,i);return r=.5,o=Math.floor((angular.element(S).width()-p.graph().width*r)/2),s=Math.floor((angular.element(S).height()-p.graph().height*r)/2),0!==v&&0!==b?(C.scale(v).translate(b),u.attr("transform","translate("+b+") scale("+v+")")):(C.scale(r).translate([o,s]),u.attr("transform","translate("+o+", "+s+") scale("+C.scale()+")")),C.on("zoom",function(){var e;return e=d3.event,v=e.scale,b=e.translate,u.attr("transform","translate("+b+") scale("+v+")")}),C(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})}},e.$watch(r.plan,function(e){if(e)return d()}),e.$watch(r.watermarks,function(t){if(t&&e.plan)return d()})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log",
 "amMoment","$q","$timeout",function(e,t,r,n,i,o){var s,a,l,u,c,d;return s=null,a=null,l={},c={running:[],finished:[],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLowerCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){if(!(e["end-time"]>-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t)
 {return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){retur
 n i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n<i;n++){if(o=t[n],o.id===e)return o;if(o.step_function&&(s=r(e,o.step_function)),s)return s}return null},t=i.defer(),l.job.promise.then(function(n){return function(i){var o;return o=r(e,s.plan.nodes),o.vertex=n.seekVertex(e),t.resolve(o)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,i;for(n=s.vertices,t=0,r=n.length;t<r;t++)if(i=n[t],i.id===e)return i;return null},this.getVertex=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(o){var a;return a=i.seekVertex(r),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return a.subtasks=e.subtasks,n.resolve(a)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r).success(function(e)
 {var t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return console.log(s.jid),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/accumulators").success(function(i){var o;return o=i["user-accumulators"],e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:o,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.re
 solve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints").success(function(e,t,n,i){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var o;return o=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?o.resolve(null):o.resolve(e)})}}(this)),o.promise},this.getOperatorBackPressure=function(r){var n;return n=i.defer(),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/b
 ackpressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPressureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"danger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/exceptions").success(function(e){return s.exceptions=e,r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel")},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metri
 c.size != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {active: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div> <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg ng-if="metric.view == \'chart\'"/> <div ng-if="metric.view != \'chart\'"> <div class="metric-numeric" title="{{value | humanizeChartNumericTitle:metric}}">{{value | humanizeChartNumeric:metric}}</div> </div> </div> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.view == \'chart\'}]" ng-click="setView(\'chart\')">Chart</button> <button type="button" ng-class="[btnClasses, {active: metric.view != \'chart\'}]" ng-click="setView(\'numeric\')">Numeric</button> </div> </div>',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:
 function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()
+}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t<Math.pow(10,n+i)?r=!0:n+=i;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.v
 alue=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-in
 terval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.f
 orEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r
 ].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s<n))return n-=1}}(this)),this.metrics[e][t].splice(n,0,r),this.saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){return _.isString(e)?{id:e,size:"small",view:"chart"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(i,o){var s;return n.setupLSFor(i,o),s=t.defer(),e.get(r.jobServer+"jobs/"+i+"/vertices/"+o+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var s;if(s=n.metrics[i][o].indexOf(e.id),s===-1&&(s=_.findIndex(n.metrics[i][o],{id:e.id})),s===-1)return t.push(e)}),s.resolve(t)}),s.promise}}(this),this.getAllAvailableMetrics=function(n){return func
 tion(n,i){var o;return o=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"/metrics").success(function(e){return o.resolve(e)}),o.promise}}(this),this.getMetrics=function(n,i,o){var s,a;return s=t.defer(),a=o.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"/metrics?get="+a).success(function(e){return function(t){var r,o;return o={},angular.forEach(t,function(e,t){return o[e.id]=parseInt(e.value)}),r={timestamp:Date.now(),values:o},e.saveValue(n,i,r),s.resolve(r)}}(this)),s.promise},this.setupLS(),this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,i){var o;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),o=n(function(){return t.loadOverv
 iew().then(function(t){return e.overview=t})},i["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(o)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadOverview=function(){var i;return i=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,o){return n=e,i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","$state","$location",function(e,t,r,n,i,o){var s;return e.yarn=o.absUrl().indexOf("/proxy/application_")!==-1,e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,null!=t.error?e.noaccess=t.error:null!=t.errors&&(e.noaccess=t.errors[0]),e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","
 submit-button":"Submit","action-time":0}},e.defaultState(),e.uploader={},e.loadList(),s=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(s)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spinner"),t.deleteJar(n).then(function(e){return angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error?alert(e.error):null!=e.errors?alert(e.errors[0]):void 0})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r,n;if("Show Plan"===e.state["plan-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.s
 tate["entry-class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["program-args"]&&(n["program-args"]=e.state["program-args"]),t.getPlan(e.state.selected,n).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",null!=t.error?e.error=t.error:null!=t.errors&&(e.error=t.errors[0]),e.plan=t.plan})["catch"](function(t){return e.state["plan-button"]="Show Plan",e.error=t})},e.runJob=function(){var r,n;if("Submit"===e.state["submit-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submitting",e.state["plan-button"]="Show Plan",e.error=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry-class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["program-args"]&&(n["program-args"]=e.state["program-args"]),e.state.savepointPath&&(n.savepointPath=e.state.savepointPath),e.state.allowNonRestoredState&&(n.allowNonRestoredState=e.state.allowNonRestoredState),t.runJob(e.state
 .selected,n).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",null!=t.error?e.error=t.error:null!=t.errors&&(e.error=t.errors[0]),null!=t.jobid))return i.go("single-job.plan.subtasks",{jobid:t.jobid})})["catch"](function(t){return e.state["submit-button"]="Submit",e.error=t})},e.nodeid=null,e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.$broadcast("reload")):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null)},e.clearFiles=function(){return e.uploader={}},e.uploadFiles=function(t){return e.uploader={},1===t.length?(e.uploader.file=t[0],e.uploader.upload=!0):e.uploader.error="Did ya forget to select a file?"},e.startUpload=function(){var t,r;return null!=e.uploader.file?(t=new FormData,t.append("jarfile",e.uploader.file),e.uploader.upload=!1,e.uploader.success="Initializing upload...",r=new XMLHttpRequest,r.upload.onprogress=function(t){return e.uploader.suc
 cess=null,e.uploader.progress=parseInt(100*t.loaded/t.total)},r.upload.onerror=function(t){return e.uploader.progress=null,e.uploader.error="An error occurred while uploading your file"},r.upload.onload=function(t){return e.uploader.progress=null,e.uploader.success="Saving..."},r.onreadystatechange=function(){var t;if(4===r.readyState)return t=JSON.parse(r.responseText),null!=t.error?(e.uploader.error=t.error,e.uploader.success=null):null!=t.errors?(e.uploader.error=t.errors[0],e.uploader.success=null):e.uploader.success="Uploaded!"},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelectClass",function(){return function(e,t){return e===t?"fa-check-square":"fa-square-o"}}),angular.module("flinkApp").service("JobSubmitService",["$http","flinkConfig","$q",function(e,t,r){return this.loadJarList=function(){var n;return n=r.defer(),e.get(t.jobServer+"jars/").success(function(e,t,r,i){return n.resolve(e)}),n.pro
 mise},this.deleteJar=function(n){var i;return i=r.defer(),e["delete"](t.jobServer+"jars/"+encodeURIComponent(n)).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.getPlan=function(n,i){var o;return o=r.defer(),e.get(t.jobServer+"jars/"+encodeURIComponent(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(e)}).error(function(e){return null!=e.e

<TRUNCATED>

[4/5] flink git commit: [FLINK-9467][metrics] Rebuild UI

Posted by ch...@apache.org.
[FLINK-9467][metrics] Rebuild UI


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

Branch: refs/heads/release-1.5
Commit: 4b2332a44b4532b3027f9e45e7f7c1131792c978
Parents: 986a377
Author: zentol <ch...@apache.org>
Authored: Tue Jun 5 13:51:29 2018 +0200
Committer: zentol <ch...@apache.org>
Committed: Thu Jun 21 16:41:45 2018 +0200

----------------------------------------------------------------------
 flink-runtime-web/web-dashboard/web/js/hs/index.js | 4 ++--
 flink-runtime-web/web-dashboard/web/js/index.js    | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[2/5] flink git commit: [FLINK-9467][metrics] Rebuild UI

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4b2332a4/flink-runtime-web/web-dashboard/web/js/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/index.js b/flink-runtime-web/web-dashboard/web/js/index.js
index ffbc7b6..ff4d3e4 100644
--- a/flink-runtime-web/web-dashboard/web/js/index.js
+++ b/flink-runtime-web/web-dashboard/web/js/index.js
@@ -1,2 +1,2 @@
-angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,o){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("overview",{url:"/overview",views:{main:{templateUrl:"partials/overview.html",controller:"OverviewController"}}}).state("runnin
 g-jobs",{url:"/running-jobs",views:{main:{templateUrl:"partials/jobs/running-jobs.html",controller:"RunningJobsController"}}}).state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{
 templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{template
 Url:"partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).sta
 te("single-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}).state("all-manager",{url:"/taskmanagers",views:{main:{templateUrl:"partials/taskmanager/index.html",controller:"AllTaskManagersController"}}}).state("single-manager",{url:"/taskmanager/{taskmanagerid}","abstract":!0,views:{main:{templateUrl:"partials/taskmanager/taskmanager.html",controller:"SingleTaskManagerController"}}}).state("single-manager.metrics",{url:"/metrics",views:{details:{templateUrl:"partials/taskmanager/taskmanager.metrics.html"}}}).state("single-manager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/taskmanager/taskmanager.stdout.html",controller:"SingleTaskManagerStdoutController"}}}).state("single-manager.log",{url:"/log",views:{details:{templateUrl:"partials/taskmanager/taskmanager.log.html",contro
 ller:"SingleTaskManagerLogsController"}}}).state("jobmanager",{url:"/jobmanager",views:{main:{templateUrl:"partials/jobmanager/index.html"}}}).state("jobmanager.config",{url:"/config",views:{details:{templateUrl:"partials/jobmanager/config.html",controller:"JobManagerConfigController"}}}).state("jobmanager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/jobmanager/stdout.html",controller:"JobManagerStdoutController"}}}).state("jobmanager.log",{url:"/log",views:{details:{templateUrl:"partials/jobmanager/log.html",controller:"JobManagerLogsController"}}}).state("submit",{url:"/submit",views:{main:{templateUrl:"partials/submit.html",controller:"JobSubmitController"}}}),t.otherwise("/overview")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLab
 elClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",
 function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,a,s;return"undefined"==typeof e||null===e?"":(i=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),o=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===o?0===a?i+"ms":a+"s ":o+"m "+a+"s":t?n+"h "+o+"m":n+"h "+o+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),e<o?(e/o).toFixed(2)+" "+r[n]:e<1e3*o?(e/o).toPrecision(3)+" "+r[n]:t(e,n+1)},"u
 ndefined"==typeof e||null===e?"":e<1e3?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filter("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}).filter("humanizeWatermark",["watermarksConfig",function(e){return function(t){return isNaN(t)||t<=e.noWatermark?"No Watermark":t}}]).filter("increment",function(){return function(e){return parseInt(e)+1}}).filter("humanizeChartNumeric",["humanizeBytesFilter","humanizeDurationFilter",function(e,t){return function(r,n){var o;return o="",null!==r&&(o=/bytes/i.test(n.id)&&/persecond/i.test(n.id)?e(r)+" / s":/bytes/i.test(n.id)?e(r):/persecond/i.test(n.id)?r+" / s":/time/i.test(n.id)||/latency/i.test(n.id)?t(r,!0):r),o}}]).filter("humanizeChartNumericTitle",["humanizeDurationFilter",function(e){return function(t,r){var n;return n="",null!==t&&(n=/bytes/i.test(r.id)&&/persecond/i.test(r.id)?t+" B
 ytes / s":/bytes/i.test(r.id)?t+" Bytes":/persecond/i.test(r.id)?t+" / s":/time/i.test(r.id)||/latency/i.test(r.id)?e(t,!1):t),n}}]).filter("searchMetrics",function(){return function(e,t){var r,n;return n=new RegExp(t,"gi"),function(){var t,o,i;for(i=[],t=0,o=e.length;t<o;t++)r=e[t],r.id.match(n)&&i.push(r);return i}()}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"config").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){r
 eturn e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval","$q","watermarksConfig",function(e,t,r,n,o,i,a,s,l,u){var c,d;return e.jobid=r.jobid,e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats={},d=s(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},a["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats=null,s.cancel(d)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.current
 Target).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})},n.loadJob(r.jobid).then(function(t){return e.job=t,e.vertices=t.vertices,e.plan=t.plan,o.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,i,a;return i=function(t){return function(t){var r,n,i,a;return r=l.defer(),i=e.job.jid,a=function(){var e,r,o;for(o=[],n=e=0,r=t.parallelism-1;0<=r?e<=r:e>=r;n=0<=r?++e:--e)o.push(n+".currentLowWatermark");return o}(),o.getMetrics(i,t.id,a).then(function(e){var t,n,o,i,a,s,l;o=NaN,l={},i=e.values;for(t in i)s=i[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(o)||s<o)&&(o=s);return n=!isNaN(o)&&o>u.noWatermark?o:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var o;return o=e.id,i(e).then(function(e){if(a[o]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.w
 atermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",fun
 ction(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},o(),e.$on("reload",function(
 e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(o(r.checkpointId),e.nodeid)return i(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["
 $scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,o;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),o=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n)
 ,e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),r<n?-1:r>n?1:0},e.dropped=function(t,n,i,a,s){return r.orderMetrics(e.jobid,e.nodeid,i,n),e.$broadcast("metrics:refresh",i),o(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),o()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),o()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),o()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),o()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return o()}),e.nodeid)return o()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{templ
 ate:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relat
 iveTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,a,s;a=r.children()[0],i=r.width(),angular.element(a).attr("width",i),s=function(e){return e.replace("&gt;",">")},o=function(r){var n,o,i;return d3.select(a).selectAll("*").remove(),i=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?i.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):i.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,o){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({
 left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(a).datum(i).call(n)},t.$watch(n.vertices,function(e){if(e)return o(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph'><g /></svg> <svg class='tmp' width='1' height='1'><g /></svg> <div class='btn-group zoom-buttons'> <a class='btn btn-default zoom-in' ng-click='zoomIn()'><i class='fa fa-plus' /></a> <a class='btn btn-default zoom-out' ng-click='zoomOut()'><i class='fa fa-minus' /></a> </div>",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,o,i,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,y,M,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomI
 n=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mi
 rror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var o,i;return o="<div href='#/jobs/"+h+"/vertex/"+e.id+"' class='node-label "+m(e,t)+"'>",o+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?o+="":(i=e.description,i=M(i),o+="<h4 class='step-name'>"+i+"</h4>"),null!=e.step_function?o+=f(e.id,r,n):(g(t)&&(o+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(o+="<h5>Parallelism: "+e.parallelism+"</h5>"),void 0!==e.lowWatermark&&(o+="<h5>Low Watermark: "+e.lowWatermark+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(o+="<h5>Operation: "+M(e.operator_strategy)+"</h5>")),o+="</div>"},f=function(e,t,r){var n,o;return o="svg-"+e,n="<svg class='"+o+"' width="+t+" height="+r+"><g /></svg>"},M=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e.length>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,o,i){return nu
 ll==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",o,i),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",o,i),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",o,i),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",o,i),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",o,i),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",o,i),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",o,i),labelType:"html","class":m(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,i,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,i=!0)
 ,a=0,u=v.length;a<u;a++)if(r=v[a],p=0,f=0,r.step_function&&(b=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),J[r.id]=b,k(b,r),g=new dagreD3.render,c.select("g").call(g,b),p=b.graph().width,f=b.graph().height,angular.element(w).empty()),s(e,t,r,i,p,f),n.push(r.id),null!=r.inputs)for(h=r.inputs,l=0,d=h.length;l<d;l++)m=h[l],o(e,t,r,n,m);return e},y=function(e,t){var r,n,o;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)for(o in r.step_function)if(r.step_function[o].id===t)return r.step_function[o]}},$=function(e,t){var r,n,o,i;if(!_.isEmpty(t))for(i=e.nodes,r=0,n=i.length;r<n;r++)o=i[r],t[o.id]&&!isNaN(t[o.id].lowWatermark)&&(o.lowWatermark=t[o.id].lowWatermark);return e},b=0,v=0,d=function(){var t,r,n,o,i,a;if(e.plan){p=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:70,edgesep:0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),k(p,$(e.plan,e.w
 atermarks)),u.selectAll("*").remove(),u.attr("transform","scale(1)"),n=new dagreD3.render,u.call(n,p);for(t in J)o=J[t],l.select("svg.svg-"+t+" g").call(n,o);return r=.5,i=Math.floor((angular.element(S).width()-p.graph().width*r)/2),a=Math.floor((angular.element(S).height()-p.graph().height*r)/2),0!==v&&0!==b?(C.scale(v).translate(b),u.attr("transform","translate("+b+") scale("+v+")")):(C.scale(r).translate([i,a]),u.attr("transform","translate("+i+", "+a+") scale("+C.scale()+")")),C.on("zoom",function(){var e;return e=d3.event,v=e.scale,b=e.translate,u.attr("transform","translate("+b+") scale("+v+")")}),C(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})}},e.$watch(r.plan,function(e){if(e)return d()}),e.$watch(r.watermarks,function(t){if(t&&e.plan)return d()})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log","amMoment","$q","$timeout",function(e,t,r,n,o,i){var a,s,l,u,c,d;return a=null,s=null,l={},c={running:[],finished:
 [],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLowerCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){if(!(e["end-time"]>-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.tim
 estamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,o,i){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,s,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),a=o,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=f
 unction(e,t){var n,o,i,a;for(n=0,o=t.length;n<o;n++){if(i=t[n],i.id===e)return i;if(i.step_function&&(a=r(e,i.step_function)),a)return a}return null},t=o.defer(),l.job.promise.then(function(n){return function(o){var i;return i=r(e,a.plan.nodes),i.vertex=n.seekVertex(e),t.resolve(i)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,o;for(n=a.vertices,t=0,r=n.length;t<r;t++)if(o=n[t],o.id===e)return o;return null},this.getVertex=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(i){var s;return s=o.seekVertex(r),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return s.subtasks=e.subtasks,n.resolve(s)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r).success(function(e){var t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=o.de
 fer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return console.log(a.jid),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/accumulators").success(function(o){var i;return i=o["user-accumulators"],e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:i,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=o.defer(),l.job.promise.then(func
 tion(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints").success(function(e,t,n,o){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var i;return i=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?i.resolve(null):i.resolve(e)})}}(this)),i.promise},this.getOperatorBackPressure=function(r){var n;return n=o.defer(),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/backpressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPre
 ssureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"danger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/exceptions").success(function(e){return a.exceptions=e,r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel")},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.size != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {ac
 tive: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div> <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg ng-if="metric.view == \'chart\'"/> <div ng-if="metric.view != \'chart\'"> <div class="metric-numeric" title="{{value | humanizeChartNumericTitle:metric}}">{{value | humanizeChartNumeric:metric}}</div> </div> </div> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.view == \'chart\'}]" ng-click="setView(\'chart\')">Chart</button> <button type="button" ng-class="[btnClasses, {active: metric.view != \'chart\'}]" ng-click="setView(\'numeric\')">Numeric</button> </div> </div>',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.
 options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e));
-},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t<Math.pow(10,n+o)?r=!0:n+=o;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===
 e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegiste
 rObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},thi
 s.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:n.size,view:o}),e.saveSetup()}}
 (this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,a){if(i.id===r.id&&(o.metrics[e][t].splice(a,1),a<n))return n-=1}}(this)),this.metrics[e][t].splice(n,0,r),this.saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){return _.isString(e)?{id:e,size:"small",view:"chart"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(o,i){var a;return n.setupLSFor(o,i),a=t.defer(),e.get(r.jobServer+"jobs/"+o+"/vertices/"+i+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var a;if(a=n.metrics[o][i].indexOf(e.id),a===-1&&(a=_.findIndex(n.metrics[o][i],{id:e.id})),a===-1)return t.push(e)}),a.resolve(t)}),a.promise}}(this),this.getAllAvailableMetrics=function(n){return function(n,o){var i;return i=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics").success(function(e){return i.resolve(e)}),i.promis
 e}}(this),this.getMetrics=function(n,o,i){var a,s;return a=t.defer(),s=i.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics?get="+s).success(function(e){return function(t){var r,i;return i={},angular.forEach(t,function(e,t){return i[e.id]=parseInt(e.value)}),r={timestamp:Date.now(),values:i},e.saveValue(n,o,r),a.resolve(r)}}(this)),a.promise},this.setupLS(),this}]),angular.module("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","$state","$location",function(e,t,r,n,o,i){var a;return e.yarn=i.absUrl().indexOf("/proxy/application_")!==-1,e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,null!=t.error?e.noaccess=t.error:null!=t.errors&&(e.noaccess=t.errors[0]),e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","submit-butt
 on":"Submit","action-time":0}},e.defaultState(),e.uploader={},e.loadList(),a=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(a)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spinner"),t.deleteJar(n).then(function(e){return angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error?alert(e.error):null!=e.errors?alert(e.errors[0]):void 0})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r,n;if("Show Plan"===e.state["plan-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry
 -class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["program-args"]&&(n["program-args"]=e.state["program-args"]),t.getPlan(e.state.selected,n).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",null!=t.error?e.error=t.error:null!=t.errors&&(e.error=t.errors[0]),e.plan=t.plan})["catch"](function(t){return e.state["plan-button"]="Show Plan",e.error=t})},e.runJob=function(){var r,n;if("Submit"===e.state["submit-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submitting",e.state["plan-button"]="Show Plan",e.error=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry-class"]),e.state.parallelism&&(n.parallelism=e.state.parallelism),e.state["program-args"]&&(n["program-args"]=e.state["program-args"]),e.state.savepointPath&&(n.savepointPath=e.state.savepointPath),e.state.allowNonRestoredState&&(n.allowNonRestoredState=e.state.allowNonRestoredState),t.runJob(e.state.selected,n
 ).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",null!=t.error?e.error=t.error:null!=t.errors&&(e.error=t.errors[0]),null!=t.jobid))return o.go("single-job.plan.subtasks",{jobid:t.jobid})})["catch"](function(t){return e.state["submit-button"]="Submit",e.error=t})},e.nodeid=null,e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.$broadcast("reload")):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null)},e.clearFiles=function(){return e.uploader={}},e.uploadFiles=function(t){return e.uploader={},1===t.length?(e.uploader.file=t[0],e.uploader.upload=!0):e.uploader.error="Did ya forget to select a file?"},e.startUpload=function(){var t,r;return null!=e.uploader.file?(t=new FormData,t.append("jarfile",e.uploader.file),e.uploader.upload=!1,e.uploader.success="Initializing upload...",r=new XMLHttpRequest,r.upload.onprogress=function(t){return e.uploader.success=null,e
 .uploader.progress=parseInt(100*t.loaded/t.total)},r.upload.onerror=function(t){return e.uploader.progress=null,e.uploader.error="An error occurred while uploading your file"},r.upload.onload=function(t){return e.uploader.progress=null,e.uploader.success="Saving..."},r.onreadystatechange=function(){var t;if(4===r.readyState)return t=JSON.parse(r.responseText),null!=t.error?(e.uploader.error=t.error,e.uploader.success=null):null!=t.errors?(e.uploader.error=t.errors[0],e.uploader.success=null):e.uploader.success="Uploaded!"},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelectClass",function(){return function(e,t){return e===t?"fa-check-square":"fa-square-o"}}),angular.module("flinkApp").service("JobSubmitService",["$http","flinkConfig","$q",function(e,t,r){return this.loadJarList=function(){var n;return n=r.defer(),e.get(t.jobServer+"jars/").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this.
 deleteJar=function(n){var o;return o=r.defer(),e["delete"](t.jobServer+"jars/"+encodeURIComponent(n)).success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.getPlan=function(n,o){var i;return i=r.defer(),e.get(t.jobServer+"jars/"+encodeURIComponent(n)+"/plan",{params:o}).success(function(e,t,r,n){return i.resolve(e)}).error(function(e){return null!=e.errors?i.reject(e.errors[0]):i.reject(e)}),i.promise},this.runJob=function(n,o){var i;return i=r.defer(),e.post(t.jobServer+"jars/"+encodeURIComponent(n)+"/run",{},{params:o}).success(function(e,t,r,n){return i.resolve(e)}).error(function(e){return null!=e.errors?i.reject(e.errors[0]):i.reject(e)}),i.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){re
 turn t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadData=function(){return t.loadLogs().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,o){return o=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,i){return n=e,o.res
 olve(e)}),o.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadStdout=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),i=n(function(){return t.loadOverview().then(function(t){return e.overview=t})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;return n=
 {},this.loadOverview=function(){var o;return o=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("AllTaskManagersController",["$scope","TaskManagersService","$interval","flinkConfig",function(e,t,r,n){var o;return t.loadManagers().then(function(t){return e.managers=t}),o=r(function(){return t.loadManagers().then(function(t){return e.managers=t})},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(o)})}]).controller("SingleTaskManagerController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.metrics={},r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t}),i=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]).controller("SingleTaskManagerLogsController",["$scope","$stateParams","Si
 ngleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.log={},e.taskmanagerid=t.taskmanagerid,r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t}),e.reloadData=function(){return r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t})}}]).controller("SingleTaskManagerStdoutController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.stdout={},e.taskmanagerid=t.taskmanagerid,r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t}),e.reloadData=function(){return r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t})}}]),angular.module("flinkApp").service("TaskManagersService",["$http","flinkConfig","$q",function(e,t,r){return this.loadManagers=function(){var n;return n=r.defer(),e.get(t.jobServer+"taskmanagers").success(function(e,t,r,o){return n.resolve(e.taskmanagers)}),n.promise},this}]).service("SingleTaskManagerService",["$http","flinkConfig","$q",function(e,t,r){
 return this.loadMetrics=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n).success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.loadLogs=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/log").success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.loadStdout=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/stdout").success(function(e,t,r,n){return o.resolve(e)}),o.promise},this}]);
\ No newline at end of file
+angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,o){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("overview",{url:"/overview",views:{main:{templateUrl:"partials/overview.html",controller:"OverviewController"}}}).state("runnin
 g-jobs",{url:"/running-jobs",views:{main:{templateUrl:"partials/jobs/running-jobs.html",controller:"RunningJobsController"}}}).state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{
 templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{template
 Url:"partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).sta
 te("single-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}).state("all-manager",{url:"/taskmanagers",views:{main:{templateUrl:"partials/taskmanager/index.html",controller:"AllTaskManagersController"}}}).state("single-manager",{url:"/taskmanager/{taskmanagerid}","abstract":!0,views:{main:{templateUrl:"partials/taskmanager/taskmanager.html",controller:"SingleTaskManagerController"}}}).state("single-manager.metrics",{url:"/metrics",views:{details:{templateUrl:"partials/taskmanager/taskmanager.metrics.html"}}}).state("single-manager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/taskmanager/taskmanager.stdout.html",controller:"SingleTaskManagerStdoutController"}}}).state("single-manager.log",{url:"/log",views:{details:{templateUrl:"partials/taskmanager/taskmanager.log.html",contro
 ller:"SingleTaskManagerLogsController"}}}).state("jobmanager",{url:"/jobmanager",views:{main:{templateUrl:"partials/jobmanager/index.html"}}}).state("jobmanager.config",{url:"/config",views:{details:{templateUrl:"partials/jobmanager/config.html",controller:"JobManagerConfigController"}}}).state("jobmanager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/jobmanager/stdout.html",controller:"JobManagerStdoutController"}}}).state("jobmanager.log",{url:"/log",views:{details:{templateUrl:"partials/jobmanager/log.html",controller:"JobManagerLogsController"}}}).state("submit",{url:"/submit",views:{main:{templateUrl:"partials/submit.html",controller:"JobSubmitController"}}}),t.otherwise("/overview")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLab
 elClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",
 function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,a,s;return"undefined"==typeof e||null===e?"":(i=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),o=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===o?0===a?i+"ms":a+"s ":o+"m "+a+"s":t?n+"h "+o+"m":n+"h "+o+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),e<o?(e/o).toFixed(2)+" "+r[n]:e<1e3*o?(e/o).toPrecision(3)+" "+r[n]:t(e,n+1)},"u
 ndefined"==typeof e||null===e?"":e<1e3?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filter("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}).filter("humanizeWatermark",["watermarksConfig",function(e){return function(t){return isNaN(t)||t<=e.noWatermark?"No Watermark":t}}]).filter("increment",function(){return function(e){return parseInt(e)+1}}).filter("humanizeChartNumeric",["humanizeBytesFilter","humanizeDurationFilter",function(e,t){return function(r,n){var o;return o="",null!==r&&(o=/bytes/i.test(n.id)&&/persecond/i.test(n.id)?e(r)+" / s":/bytes/i.test(n.id)?e(r):/persecond/i.test(n.id)?r+" / s":/time/i.test(n.id)||/latency/i.test(n.id)?t(r,!0):r),o}}]).filter("humanizeChartNumericTitle",["humanizeDurationFilter",function(e){return function(t,r){var n;return n="",null!==t&&(n=/bytes/i.test(r.id)&&/persecond/i.test(r.id)?t+" B
 ytes / s":/bytes/i.test(r.id)?t+" Bytes":/persecond/i.test(r.id)?t+" / s":/time/i.test(r.id)||/latency/i.test(r.id)?e(t,!1):t),n}}]).filter("searchMetrics",function(){return function(e,t){var r,n;return n=new RegExp(t,"gi"),function(){var t,o,i;for(i=[],t=0,o=e.length;t<o;t++)r=e[t],r.id.match(n)&&i.push(r);return i}()}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"config").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){return t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadD
 ata=function(){return t.loadLogs().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,o){return o=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var
  n;return n={},this.loadStdout=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval","$q","watermarksConfig",function(e,t,r,n,o,i,a,s,l,u){var c,d;r
 eturn e.jobid=r.jobid,e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats={},d=s(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},a["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.watermarks={},e.vertices=null,e.backPressureOperatorStats=null,s.cancel(d)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})},n.loadJob(r.jobid).then(function(t){return e.job=t,e.vertices=t.vertices,e.plan=t.plan,o.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,i,a;return i=function(t){return function(t){var r,n,i,a;return r=l.defer(),i=e.job.jid,a=function(){var e,r,o;for(o=[],n=e=0,r
 =t.parallelism-1;0<=r?e<=r:e>=r;n=0<=r?++e:--e)o.push(n+".currentInputWatermark");return o}(),o.getMetrics(i,t.id,a).then(function(e){var t,n,o,i,a,s,l;o=NaN,l={},i=e.values;for(t in i)s=i[t],a=t.replace(".currentInputWatermark",""),l[a]=s,(isNaN(o)||s<o)&&(o=s);return n=!isNaN(o)&&o>u.noWatermark?o:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var o;return o=e.id,i(e).then(function(e){if(a[o]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.s
 tateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;retu
 rn r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then
 (function(t){if(null!==t)return e.subtaskDetails[r]=t})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(o(r.checkpointId),e.nodeid)return i(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller(
 "JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,o;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),o=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),r<n?-1:r>n?1:0},e.dropped=function(t,n,i,a,s){return r.orderMetrics(e.jobid,e.nodeid,i,n),e.$broadcast("metrics:refresh",i),o(),!1},e.dragStart=fun
 ction(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),o()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),o()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),o()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),o()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return o()}),e.nodeid)return o()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SC
 HEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,a,s;a=r.children()[0],i=r.width(),angular.element(a).attr("width",i),s=function(e){return e.replace("&gt;",">")},o=function(r){var n,o,i;return d3.select(a).selec
 tAll("*").remove(),i=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?i.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):i.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,o){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(a).datum(i).call(n)},t.$watch(n.vertices,function(e){if(e)return o(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph'><g /></svg> 
 <svg class='tmp' width='1' height='1'><g /></svg> <div class='btn-group zoom-buttons'> <a class='btn btn-default zoom-in' ng-click='zoomIn()'><i class='fa fa-plus' /></a> <a class='btn btn-default zoom-out' ng-click='zoomOut()'><i class='fa fa-minus' /></a> </div>",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,o,i,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,y,M,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r
 =e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var o,i;return o="<div href='#/jobs/"+h+"/vertex/"+e.id+"' class='node-label "+m(e,t)+"'>",o+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?o+="":(i=e.description,i=M(i),o+="<h4 class='step-name'>"+i+"</h4>"),null!=e.step_function?o+=f(e.id,r,
 n):(g(t)&&(o+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(o+="<h5>Parallelism: "+e.parallelism+"</h5>"),void 0!==e.lowWatermark&&(o+="<h5>Low Watermark: "+e.lowWatermark+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(o+="<h5>Operation: "+M(e.operator_strategy)+"</h5>")),o+="</div>"},f=function(e,t,r){var n,o;return o="svg-"+e,n="<svg class='"+o+"' width="+t+" height="+r+"><g /></svg>"},M=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e.length>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,o,i){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",o,i),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",o,i),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",o,i),labelType:"html","class":m(r,"workset")}):r.id===t.next
 _workset?e.setNode(r.id,{label:a(r,"nextWorkset",o,i),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",o,i),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",o,i),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",o,i),labelType:"html","class":m(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,i,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,i=!0),a=0,u=v.length;a<u;a++)if(r=v[a],p=0,f=0,r.step_function&&(b=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),J[r.id]=b,k(b,r),g=new dagreD3.render,c.select("g").call(g,b),p=b.graph().width,f=b.graph().height,angular.element(w).empty()),s(e,t,r,i,p,f),n.push(r.id),null!=r.inputs)for(h=r.inputs,l=0,d=h.
 length;l<d;l++)m=h[l],o(e,t,r,n,m);return e},y=function(e,t){var r,n,o;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)for(o in r.step_function)if(r.step_function[o].id===t)return r.step_function[o]}},$=function(e,t){var r,n,o,i;if(!_.isEmpty(t))for(i=e.nodes,r=0,n=i.length;r<n;r++)o=i[r],t[o.id]&&!isNaN(t[o.id].lowWatermark)&&(o.lowWatermark=t[o.id].lowWatermark);return e},b=0,v=0,d=function(){var t,r,n,o,i,a;if(e.plan){p=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:70,edgesep:0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),k(p,$(e.plan,e.watermarks)),u.selectAll("*").remove(),u.attr("transform","scale(1)"),n=new dagreD3.render,u.call(n,p);for(t in J)o=J[t],l.select("svg.svg-"+t+" g").call(n,o);return r=.5,i=Math.floor((angular.element(S).width()-p.graph().width*r)/2),a=Math.floor((angular.element(S).height()-p.graph().height*r)/2),0!==v&&0!==b?(C.scale(v).translate(b),u.attr("transform","translate("+b+") scale("+v+")")):(
 C.scale(r).translate([i,a]),u.attr("transform","translate("+i+", "+a+") scale("+C.scale()+")")),C.on("zoom",function(){var e;return e=d3.event,v=e.scale,b=e.translate,u.attr("transform","translate("+b+") scale("+v+")")}),C(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})}},e.$watch(r.plan,function(e){if(e)return d()}),e.$watch(r.watermarks,function(t){if(t&&e.plan)return d()})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log","amMoment","$q","$timeout",function(e,t,r,n,o,i){var a,s,l,u,c,d;return a=null,s=null,l={},c={running:[],finished:[],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLow
 erCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){if(!(e["end-time"]>-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,o,i){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach
 (function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,s,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),a=o,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,o,i,a;for(n=0,o=t.length;n<o;n++){if(i=t[n],i.id===e)return i;if(i.step_function&&(a=r(e,i.step_function)),a)return a}return null},t=o.defer(),l.job.promise.then(function(n){return function(o){var i;return i=r(e,a.plan.nodes),i.vertex=n.seekVertex(e),t.resolve(i)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,o;for(n=a.vertices,t=0,r=n.length;t<r;t++)if(o=n[
 t],o.id===e)return o;return null},this.getVertex=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(i){var s;return s=o.seekVertex(r),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return s.subtasks=e.subtasks,n.resolve(s)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r).success(function(e){var t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return console.log(a.jid),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"
 +r+"/accumulators").success(function(o){var i;return i=o["user-accumulators"],e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:i,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints").success(function(e,t,n,o){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/details/"+r).success(function(e){retu
 rn angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var i;return i=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?i.resolve(null):i.resolve(e)})}}(this)),i.promise},this.getOperatorBackPressure=function(r){var n;return n=o.defer(),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/backpressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPressureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"danger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/exceptions").success(function(e){return a.exceptions=e,
 r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel");
+},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.size != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {active: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div> <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg ng-if="metric.view == \'chart\'"/> <div ng-if="metric.view != \'chart\'"> <div class="metric-numeric" title="{{value | humanizeChartNumericTitle:metric}}">{{value | humanizeChartNumeric:metric}}</div> </div> </div> <div class="buttons"> <div class="btn-group"> <butt
 on type="button" ng-class="[btnClasses, {active: metric.view == \'chart\'}]" ng-click="setView(\'chart\')">Chart</button> <button type="button" ng-class="[btnClasses, {active: metric.view != \'chart\'}]" ng-click="setView(\'numeric\')">Numeric</button> </div> </div>',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t<Math.pow(10,n+o)?r=!0:n+=o;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,lef
 t:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r
 ,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.sav
 eSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[
 t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:n.size,view:o}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,a){if(i.id===r.id&&(o.metrics[e][t].splice(a,1),a<n))return n-=1}}(this)),this.metrics[e][t].splice(n,0,r),this.saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){retur
 n _.isString(e)?{id:e,size:"small",view:"chart"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(o,i){var a;return n.setupLSFor(o,i),a=t.defer(),e.get(r.jobServer+"jobs/"+o+"/vertices/"+i+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var a;if(a=n.metrics[o][i].indexOf(e.id),a===-1&&(a=_.findIndex(n.metrics[o][i],{id:e.id})),a===-1)return t.push(e)}),a.resolve(t)}),a.promise}}(this),this.getAllAvailableMetrics=function(n){return function(n,o){var i;return i=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics").success(function(e){return i.resolve(e)}),i.promise}}(this),this.getMetrics=function(n,o,i){var a,s;return a=t.defer(),s=i.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics?get="+s).success(function(e){return function(t){var r,i;return i={},angular.forEach(t,function(e,t){return i[e.id]=parseInt(e.value)}),r={timestamp:Date.now(),values:i},e.saveValue(n,o,r),a.resolve(r)}}(this)),a.promise},this.setup
 LS(),this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),i=n(function(){return t.loadOverview().then(function(t){return e.overview=t})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadOverview=function(){var o;return o=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","
 $state","$location",function(e,t,r,n,o,i){var a;return e.yarn=i.absUrl().indexOf("/proxy/application_")!==-1,e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,null!=t.error?e.noaccess=t.error:null!=t.errors&&(e.noaccess=t.errors[0]),e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","submit-button":"Submit","action-time":0}},e.defaultState(),e.uploader={},e.loadList(),a=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(a)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spinner"),t.deleteJar(n).then(function(e){return angul
 ar.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error?alert(e.error):null!=e.errors?alert(e.errors[0]):void 0})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r,n;if("Show Plan"===e.state["plan-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,n={},e.state["entry-class"]&&(n["entry-class"]=e.state["entry-class"]),e.state.parallelism&&(n.parallelism=

<TRUNCATED>