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 2017/07/04 17:10:19 UTC

[1/9] flink git commit: [FLINK-6008] Collection of BlobServer improvements

Repository: flink
Updated Branches:
  refs/heads/master 85853edab -> 4d18afed8


[FLINK-6008] Collection of BlobServer improvements

- [docs] update some config options to the new, non-deprecated ones
- [docs] improvements in the BlobService docs

- use Preconditions.checkArgument in BlobClient
- refactor BlobCache#getURL()
- do not fail the BlobServer if delete fails
- fix concurrent job directory creation
- do not guard a delete() call with a check for existence
- cleanup javadocs in (Blob)LibraryCacheManager
- cleanup tests for BlobLibraryCacheManager

This closes #4146.


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

Branch: refs/heads/master
Commit: 28c3354e735e98657d8d90912234cf8e896e78d6
Parents: 85853ed
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Fri Jan 6 18:42:58 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 14:57:25 2017 +0200

----------------------------------------------------------------------
 docs/setup/config.md                            |   6 +-
 .../apache/flink/runtime/blob/BlobCache.java    |  82 +++------
 .../apache/flink/runtime/blob/BlobClient.java   |  18 +-
 .../apache/flink/runtime/blob/BlobServer.java   |   6 +-
 .../runtime/blob/BlobServerConnection.java      |  18 +-
 .../apache/flink/runtime/blob/BlobService.java  |   7 +-
 .../apache/flink/runtime/blob/BlobUtils.java    |   4 +-
 .../librarycache/BlobLibraryCacheManager.java   |  22 +++
 .../librarycache/LibraryCacheManager.java       |  24 ++-
 .../runtime/blob/BlobServerDeleteTest.java      | 165 ++++++++++++++++---
 .../flink/runtime/blob/BlobServerPutTest.java   | 105 ++++++++++++
 .../BlobLibraryCacheManagerTest.java            | 145 ++++++++++++++--
 12 files changed, 464 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index 1511f34..b4bf845 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -510,6 +510,8 @@ May be set to -1 to disable this feature.
   - `none` (default): No high availability. A single JobManager runs and no JobManager state is checkpointed.
   - `zookeeper`: Supports the execution of multiple JobManagers and JobManager state checkpointing. Among the group of JobManagers, ZooKeeper elects one of them as the leader which is responsible for the cluster execution. In case of a JobManager failure, a standby JobManager will be elected as the new leader and is given the last checkpointed JobManager state. In order to use the 'zookeeper' mode, it is mandatory to also define the `high-availability.zookeeper.quorum` configuration value.
 
+- `high-availability.cluster-id`: (Default `/default_ns` in standalone cluster mode, or the <yarn-application-id> under YARN) Defines the subdirectory under the root dir where the ZooKeeper HA mode will create znodes. This allows to isolate multiple applications on the same ZooKeeper. Previously this key was named `recovery.zookeeper.path.namespace` and `high-availability.zookeeper.path.namespace`.
+
 Previously this key was named `recovery.mode` and the default value was `standalone`.
 
 #### ZooKeeper-based HA Mode
@@ -518,13 +520,11 @@ Previously this key was named `recovery.mode` and the default value was `standal
 
 - `high-availability.zookeeper.path.root`: (Default `/flink`) Defines the root dir under which the ZooKeeper HA mode will create namespace directories. Previously this ket was named `recovery.zookeeper.path.root`.
 
-- `high-availability.zookeeper.path.namespace`: (Default `/default_ns` in standalone cluster mode, or the <yarn-application-id> under YARN) Defines the subdirectory under the root dir where the ZooKeeper HA mode will create znodes. This allows to isolate multiple applications on the same ZooKeeper. Previously this key was named `recovery.zookeeper.path.namespace`.
-
 - `high-availability.zookeeper.path.latch`: (Default `/leaderlatch`) Defines the znode of the leader latch which is used to elect the leader. Previously this key was named `recovery.zookeeper.path.latch`.
 
 - `high-availability.zookeeper.path.leader`: (Default `/leader`) Defines the znode of the leader which contains the URL to the leader and the current leader session ID. Previously this key was named `recovery.zookeeper.path.leader`.
 
-- `high-availability.zookeeper.storageDir`: Defines the directory in the state backend where the JobManager metadata will be stored (ZooKeeper only keeps pointers to it). Required for HA. Previously this key was named `recovery.zookeeper.storageDir`.
+- `high-availability.storageDir`: Defines the directory in the state backend where the JobManager metadata will be stored (ZooKeeper only keeps pointers to it). Required for HA. Previously this key was named `recovery.zookeeper.storageDir` and `high-availability.zookeeper.storageDir`.
 
 - `high-availability.zookeeper.client.session-timeout`: (Default `60000`) Defines the session timeout for the ZooKeeper session in ms. Previously this key was named `recovery.zookeeper.client.session-timeout`
 

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
index aa47eae..32bd8fd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.blob;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -142,78 +141,45 @@ public final class BlobCache implements BlobService {
 
 		// fallback: download from the BlobServer
 		final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE];
+		LOG.info("Downloading {} from {}", requiredBlob, serverAddress);
 
 		// loop over retries
 		int attempt = 0;
 		while (true) {
-
-			if (attempt == 0) {
-				LOG.info("Downloading {} from {}", requiredBlob, serverAddress);
-			} else {
-				LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt);
-			}
-
-			try {
-				BlobClient bc = null;
-				InputStream is = null;
-				OutputStream os = null;
-
-				try {
-					bc = new BlobClient(serverAddress, blobClientConfig);
-					is = bc.get(requiredBlob);
-					os = new FileOutputStream(localJarFile);
-
-					while (true) {
-						final int read = is.read(buf);
-						if (read < 0) {
-							break;
-						}
-						os.write(buf, 0, read);
-					}
-
-					// we do explicitly not use a finally block, because we want the closing
-					// in the regular case to throw exceptions and cause the writing to fail.
-					// But, the closing on exception should not throw further exceptions and
-					// let us keep the root exception
-					os.close();
-					os = null;
-					is.close();
-					is = null;
-					bc.close();
-					bc = null;
-
-					// success, we finished
-					return localJarFile.toURI().toURL();
-				}
-				catch (Throwable t) {
-					// we use "catch (Throwable)" to keep the root exception. Otherwise that exception
-					// it would be replaced by any exception thrown in the finally block
-					IOUtils.closeQuietly(os);
-					IOUtils.closeQuietly(is);
-					IOUtils.closeQuietly(bc);
-
-					if (t instanceof IOException) {
-						throw (IOException) t;
-					} else {
-						throw new IOException(t.getMessage(), t);
+			try (
+				final BlobClient bc = new BlobClient(serverAddress, blobClientConfig);
+				final InputStream is = bc.get(requiredBlob);
+				final OutputStream os = new FileOutputStream(localJarFile)
+			) {
+				while (true) {
+					final int read = is.read(buf);
+					if (read < 0) {
+						break;
 					}
+					os.write(buf, 0, read);
 				}
+
+				// success, we finished
+				return localJarFile.toURI().toURL();
 			}
-			catch (IOException e) {
+			catch (Throwable t) {
 				String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress +
 					" and store it under " + localJarFile.getAbsolutePath();
 				if (attempt < numFetchRetries) {
-					attempt++;
 					if (LOG.isDebugEnabled()) {
-						LOG.debug(message + " Retrying...", e);
+						LOG.debug(message + " Retrying...", t);
 					} else {
 						LOG.error(message + " Retrying...");
 					}
 				}
 				else {
-					LOG.error(message + " No retries left.", e);
-					throw new IOException(message, e);
+					LOG.error(message + " No retries left.", t);
+					throw new IOException(message, t);
 				}
+
+				// retry
+				++attempt;
+				LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt);
 			}
 		} // end loop over retries
 	}
@@ -225,8 +191,8 @@ public final class BlobCache implements BlobService {
 	public void delete(BlobKey key) throws IOException{
 		final File localFile = BlobUtils.getStorageLocation(storageDir, key);
 
-		if (localFile.exists() && !localFile.delete()) {
-			LOG.warn("Failed to delete locally cached BLOB " + key + " at " + localFile.getAbsolutePath());
+		if (!localFile.delete() && localFile.exists()) {
+			LOG.warn("Failed to delete locally cached BLOB {} at {}", key, localFile.getAbsolutePath());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
index fab3c5c..0f02e75 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
@@ -64,6 +64,7 @@ import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY;
 import static org.apache.flink.runtime.blob.BlobUtils.readFully;
 import static org.apache.flink.runtime.blob.BlobUtils.readLength;
 import static org.apache.flink.runtime.blob.BlobUtils.writeLength;
+import static org.apache.flink.util.Preconditions.checkArgument;
 
 /**
  * The BLOB client can communicate with the BLOB server and either upload (PUT), download (GET),
@@ -593,9 +594,7 @@ public final class BlobClient implements Closeable {
 	 *         the BLOB server or if the BLOB server cannot delete the file
 	 */
 	public void delete(BlobKey key) throws IOException {
-		if (key == null) {
-			throw new IllegalArgumentException("BLOB key must not be null");
-		}
+		checkArgument(key != null, "BLOB key must not be null.");
 
 		deleteInternal(null, null, key);
 	}
@@ -611,12 +610,9 @@ public final class BlobClient implements Closeable {
 	 *         thrown if an I/O error occurs while transferring the request to the BLOB server
 	 */
 	public void delete(JobID jobId, String key) throws IOException {
-		if (jobId == null) {
-			throw new IllegalArgumentException("JobID must not be null");
-		}
-		if (key == null) {
-			throw new IllegalArgumentException("Key must not be null");
-		}
+		checkArgument(jobId != null, "Job id must not be null.");
+		checkArgument(key != null, "BLOB name must not be null.");
+		
 		if (key.length() > MAX_KEY_LENGTH) {
 			throw new IllegalArgumentException("Keys must not be longer than " + MAX_KEY_LENGTH);
 		}
@@ -633,9 +629,7 @@ public final class BlobClient implements Closeable {
 	 *         thrown if an I/O error occurs while transferring the request to the BLOB server
 	 */
 	public void deleteAll(JobID jobId) throws IOException {
-		if (jobId == null) {
-			throw new IllegalArgumentException("Argument jobID must not be null");
-		}
+		checkArgument(jobId != null, "Job id must not be null.");
 
 		deleteInternal(jobId, null, null);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index 5ad4b6a..3b3a39b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -411,10 +411,8 @@ public class BlobServer extends Thread implements BlobService {
 		readWriteLock.writeLock().lock();
 
 		try {
-			if (localFile.exists()) {
-				if (!localFile.delete()) {
-					LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath());
-				}
+			if (!localFile.delete() && localFile.exists()) {
+				LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath());
 			}
 
 			blobStore.delete(key);

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
index a76dbd5..ad33c5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
@@ -509,21 +509,7 @@ class BlobServerConnection extends Thread {
 
 			if (type == CONTENT_ADDRESSABLE) {
 				BlobKey key = BlobKey.readFromInputStream(inputStream);
-				File blobFile = blobServer.getStorageLocation(key);
-
-				writeLock.lock();
-
-				try {
-					// we should make the local and remote file deletion atomic, otherwise we might risk not
-					// removing the remote file in case of a concurrent put operation
-					if (blobFile.exists() && !blobFile.delete()) {
-						throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath());
-					}
-
-					blobStore.delete(key);
-				} finally {
-					writeLock.unlock();
-				}
+				blobServer.delete(key);
 			}
 			else if (type == NAME_ADDRESSABLE) {
 				byte[] jidBytes = new byte[JobID.SIZE];
@@ -540,7 +526,7 @@ class BlobServerConnection extends Thread {
 					// we should make the local and remote file deletion atomic, otherwise we might risk not
 					// removing the remote file in case of a concurrent put operation
 					if (blobFile.exists() && !blobFile.delete()) {
-						throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath());
+						LOG.warn("Cannot delete local BLOB file " + blobFile.getAbsolutePath());
 					}
 
 					blobStore.delete(jobID, key);

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
index 97a2d51..c1447c8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
@@ -28,17 +28,18 @@ import java.net.URL;
 public interface BlobService extends Closeable {
 
 	/**
-	 * This method returns the URL of the file associated with the provided blob key.
+	 * Returns the URL of the file associated with the provided blob key.
 	 *
 	 * @param key blob key associated with the requested file
 	 * @return The URL to the file.
-	 * @throws IOException
+	 * @throws java.io.FileNotFoundException when the path does not exist;
+	 * @throws IOException if any other error occurs when retrieving the file
 	 */
 	URL getURL(BlobKey key) throws IOException;
 
 
 	/**
-	 * This method deletes the file associated with the provided blob key.
+	 * Deletes the file associated with the provided blob key.
 	 *
 	 * @param key associated with the file to be deleted
 	 * @throws IOException

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index 8da362d..9d538cc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -227,7 +227,9 @@ public class BlobUtils {
 	private static File getJobDirectory(File storageDir, JobID jobID) {
 		final File jobDirectory = new File(storageDir, JOB_DIR_PREFIX + jobID.toString());
 
-		if (!jobDirectory.exists() && !jobDirectory.mkdirs()) {
+		// note: thread-safe create should try to mkdir first and then ignore the case that the
+		//       directory already existed
+		if (!jobDirectory.mkdirs() && !jobDirectory.exists()) {
 			throw new RuntimeException("Could not create jobId directory '" + jobDirectory.getAbsolutePath() + "'.");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
index 0702a11..0c4cb85 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
@@ -48,6 +48,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * a set of libraries (typically JAR files) which the job requires to run. The library cache manager
  * caches library files in order to avoid unnecessary retransmission of data. It is based on a singleton
  * programming pattern, so there exists at most one library manager at a time.
+ * <p>
+ * All files registered via {@link #registerJob(JobID, Collection, Collection)} are reference-counted
+ * and are removed by a timer-based cleanup task if their reference counter is zero.
+ * <strong>NOTE:</strong> this does not apply to files that enter the blob service via
+ * {@link #getFile(BlobKey)}!
  */
 public final class BlobLibraryCacheManager extends TimerTask implements LibraryCacheManager {
 
@@ -73,6 +78,12 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC
 	
 	// --------------------------------------------------------------------------------------------
 
+	/**
+	 * Creates the blob library cache manager.
+	 *
+	 * @param blobService blob file retrieval service to use
+	 * @param cleanupInterval cleanup interval in milliseconds
+	 */
 	public BlobLibraryCacheManager(BlobService blobService, long cleanupInterval) {
 		this.blobService = checkNotNull(blobService);
 
@@ -191,6 +202,17 @@ public final class BlobLibraryCacheManager extends TimerTask implements LibraryC
 		}
 	}
 
+	/**
+	 * Returns a file handle to the file identified by the blob key.
+	 * <p>
+	 * <strong>NOTE:</strong> if not already registered during
+	 * {@link #registerJob(JobID, Collection, Collection)}, files that enter the library cache /
+	 * backing blob store using this method will not be reference-counted and garbage-collected!
+	 *
+	 * @param blobKey identifying the requested file
+	 * @return File handle
+	 * @throws IOException if any error occurs when retrieving the file
+	 */
 	@Override
 	public File getFile(BlobKey blobKey) throws IOException {
 		return new File(blobService.getURL(blobKey).getFile());

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
index 03c8f27..bf05271 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
@@ -41,7 +41,7 @@ public interface LibraryCacheManager {
 	 *
 	 * @param blobKey identifying the requested file
 	 * @return File handle
-	 * @throws IOException
+	 * @throws IOException if any error occurs when retrieving the file
 	 */
 	File getFile(BlobKey blobKey) throws IOException;
 
@@ -51,7 +51,9 @@ public interface LibraryCacheManager {
 	 * @param id job ID
 	 * @param requiredJarFiles collection of blob keys identifying the required jar files
 	 * @param requiredClasspaths collection of classpaths that are added to the user code class loader
-	 * @throws IOException
+	 * @throws IOException if any error occurs when retrieving the required jar files
+	 *
+	 * @see #unregisterJob(JobID) counterpart of this method
 	 */
 	void registerJob(JobID id, Collection<BlobKey> requiredJarFiles, Collection<URL> requiredClasspaths)
 			throws IOException;
@@ -63,21 +65,35 @@ public interface LibraryCacheManager {
 	 * @param requiredJarFiles collection of blob keys identifying the required jar files
 	 * @param requiredClasspaths collection of classpaths that are added to the user code class loader
 	 * @throws IOException
+	 *
+	 * @see #unregisterTask(JobID, ExecutionAttemptID) counterpart of this method
 	 */
 	void registerTask(JobID id, ExecutionAttemptID execution, Collection<BlobKey> requiredJarFiles,
 			Collection<URL> requiredClasspaths) throws IOException;
 
 	/**
-	 * Unregisters a job from the library cache manager.
+	 * Unregisters a job task execution from the library cache manager.
+	 * <p>
+	 * <strong>Note:</strong> this is the counterpart of {@link #registerTask(JobID,
+	 * ExecutionAttemptID, Collection, Collection)} and it will not remove any job added via
+	 * {@link #registerJob(JobID, Collection, Collection)}!
 	 *
 	 * @param id job ID
+	 *
+	 * @see #registerTask(JobID, ExecutionAttemptID, Collection, Collection) counterpart of this method
 	 */
 	void unregisterTask(JobID id, ExecutionAttemptID execution);
-	
+
 	/**
 	 * Unregisters a job from the library cache manager.
+	 * <p>
+	 * <strong>Note:</strong> this is the counterpart of {@link #registerJob(JobID, Collection,
+	 * Collection)} and it will not remove any job task execution added via {@link
+	 * #registerTask(JobID, ExecutionAttemptID, Collection, Collection)}!
 	 *
 	 * @param id job ID
+	 *
+	 * @see #registerJob(JobID, Collection, Collection) counterpart of this method
 	 */
 	void unregisterJob(JobID id);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
index 5e1d86e..15e2c7a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
@@ -39,6 +39,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -53,7 +54,7 @@ public class BlobServerDeleteTest extends TestLogger {
 	private final Random rnd = new Random();
 
 	@Test
-	public void testDeleteSingle() {
+	public void testDeleteSingleByBlobKey() {
 		BlobServer server = null;
 		BlobClient client = null;
 		BlobStore blobStore = new VoidBlobStore();
@@ -72,7 +73,13 @@ public class BlobServerDeleteTest extends TestLogger {
 			BlobKey key = client.put(data);
 			assertNotNull(key);
 
-			// issue a DELETE request
+			// second item
+			data[0] ^= 1;
+			BlobKey key2 = client.put(data);
+			assertNotNull(key2);
+			assertNotEquals(key, key2);
+
+			// issue a DELETE request via the client
 			client.delete(key);
 			client.close();
 
@@ -92,6 +99,69 @@ public class BlobServerDeleteTest extends TestLogger {
 			catch (IllegalStateException e) {
 				// expected
 			}
+
+			// delete a file directly on the server
+			server.delete(key2);
+			try {
+				server.getURL(key2);
+				fail("BLOB should have been deleted");
+			}
+			catch (IOException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			cleanup(server, client);
+		}
+	}
+
+	@Test
+	public void testDeleteSingleByName() {
+		BlobServer server = null;
+		BlobClient client = null;
+		BlobStore blobStore = new VoidBlobStore();
+
+		try {
+			Configuration config = new Configuration();
+			server = new BlobServer(config, blobStore);
+
+			InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort());
+			client = new BlobClient(serverAddress, config);
+
+			byte[] data = new byte[2000000];
+			rnd.nextBytes(data);
+
+			JobID jobID = new JobID();
+			String name1 = "random name";
+			String name2 = "any nyme";
+
+			client.put(jobID, name1, data);
+			client.put(jobID, name2, data);
+
+			// issue a DELETE request via the client
+			client.delete(jobID, name1);
+			client.close();
+
+			client = new BlobClient(serverAddress, config);
+			try {
+				client.get(jobID, name1);
+				fail("BLOB should have been deleted");
+			}
+			catch (IOException e) {
+				// expected
+			}
+
+			try {
+				client.put(new byte[1]);
+				fail("client should be closed after erroneous operation");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -125,9 +195,15 @@ public class BlobServerDeleteTest extends TestLogger {
 			// put content addressable (like libraries)
 			client.put(jobID, name1, data);
 			client.put(jobID, name2, new byte[712]);
+			// items for a second (different!) job ID
+			final byte[] jobIdBytes = jobID.getBytes();
+			jobIdBytes[0] ^= 1;
+			JobID jobID2 = JobID.fromByteArray(jobIdBytes);
+			client.put(jobID2, name1, data);
+			client.put(jobID2, name2, new byte[712]);
 
 
-			// issue a DELETE ALL request
+			// issue a DELETE ALL request via the client
 			client.deleteAll(jobID);
 			client.close();
 
@@ -189,13 +265,16 @@ public class BlobServerDeleteTest extends TestLogger {
 			File blobFile = server.getStorageLocation(key);
 			assertTrue(blobFile.delete());
 
-			// issue a DELETE request
+			// issue a DELETE request via the client
 			try {
 				client.delete(key);
 			}
 			catch (IOException e) {
 				fail("DELETE operation should not fail if file is already deleted");
 			}
+
+			// issue a DELETE request on the server
+			server.delete(key);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -230,7 +309,7 @@ public class BlobServerDeleteTest extends TestLogger {
 			File blobFile = server.getStorageLocation(jid, name);
 			assertTrue(blobFile.delete());
 
-			// issue a DELETE request
+			// issue a DELETE request via the client
 			try {
 				client.delete(jid, name);
 			}
@@ -248,13 +327,15 @@ public class BlobServerDeleteTest extends TestLogger {
 	}
 
 	@Test
-	public void testDeleteFails() {
+	public void testDeleteFailsByBlobKey() {
 		assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows.
 
 		BlobServer server = null;
 		BlobClient client = null;
 		BlobStore blobStore = new VoidBlobStore();
 
+		File blobFile = null;
+		File directory = null;
 		try {
 			Configuration config = new Configuration();
 			server = new BlobServer(config, blobStore);
@@ -269,30 +350,74 @@ public class BlobServerDeleteTest extends TestLogger {
 			BlobKey key = client.put(data);
 			assertNotNull(key);
 
-			File blobFile = server.getStorageLocation(key);
-			File directory = blobFile.getParentFile();
+			blobFile = server.getStorageLocation(key);
+			directory = blobFile.getParentFile();
 
 			assertTrue(blobFile.setWritable(false, false));
 			assertTrue(directory.setWritable(false, false));
 
-			// issue a DELETE request
-			try {
-				client.delete(key);
-				fail("DELETE operation should fail if file cannot be deleted");
-			}
-			catch (IOException e) {
-				// expected
-			}
-			finally {
+			// issue a DELETE request via the client
+			client.delete(key);
+
+			// issue a DELETE request on the server
+			server.delete(key);
+
+			// the file should still be there
+			server.getURL(key);
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		} finally {
+			if (blobFile != null && directory != null) {
 				blobFile.setWritable(true, false);
 				directory.setWritable(true, false);
 			}
+			cleanup(server, client);
 		}
-		catch (Exception e) {
+	}
+
+	@Test
+	public void testDeleteByNameFails() {
+		assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows.
+
+		BlobServer server = null;
+		BlobClient client = null;
+		BlobStore blobStore = new VoidBlobStore();
+
+		File blobFile = null;
+		File directory = null;
+		try {
+			Configuration config = new Configuration();
+			server = new BlobServer(config, blobStore);
+
+			InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort());
+			client = new BlobClient(serverAddress, config);
+
+			byte[] data = new byte[2000000];
+			rnd.nextBytes(data);
+
+			JobID jid = new JobID();
+			String name = "------------fdghljEgRJHF+##4U789Q345";
+
+			client.put(jid, name, data);
+
+			blobFile = server.getStorageLocation(jid, name);
+			directory = blobFile.getParentFile();
+
+			assertTrue(blobFile.setWritable(false, false));
+			assertTrue(directory.setWritable(false, false));
+
+			// issue a DELETE request via the client
+			client.delete(jid, name);
+
+			// the file should still be there
+			client.get(jid, name);
+		} catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
-		}
-		finally {
+		} finally {
+			blobFile.setWritable(true, false);
+			directory.setWritable(true, false);
 			cleanup(server, client);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
index 35ef968..31e63a0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.blob;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
@@ -59,6 +60,110 @@ public class BlobServerPutTest extends TestLogger {
 
 	private final Random rnd = new Random();
 
+
+	// --- concurrency tests for utility methods which could fail during the put operation ---
+
+	/**
+	 * Checked thread that calls {@link BlobServer#getStorageLocation(BlobKey)}
+	 */
+	public static class ContentAddressableGetStorageLocation extends CheckedThread {
+		private final BlobServer server;
+		private final BlobKey key;
+
+		public ContentAddressableGetStorageLocation(BlobServer server, BlobKey key) {
+			this.server = server;
+			this.key = key;
+		}
+
+		@Override
+		public void go() throws Exception {
+			server.getStorageLocation(key);
+		}
+	}
+
+	/**
+	 * Tests concurrent calls to {@link BlobServer#getStorageLocation(BlobKey)}.
+	 */
+	@Test
+	public void testServerContentAddressableGetStorageLocationConcurrent() throws Exception {
+		BlobServer server = new BlobServer(new Configuration(), new VoidBlobStore());
+
+		try {
+			BlobKey key = new BlobKey();
+			CheckedThread[] threads = new CheckedThread[] {
+				new ContentAddressableGetStorageLocation(server, key),
+				new ContentAddressableGetStorageLocation(server, key),
+				new ContentAddressableGetStorageLocation(server, key)
+			};
+			checkedThreadSimpleTest(threads);
+		} finally {
+			server.close();
+		}
+	}
+
+	/**
+	 * Helper method to first start all threads and then wait for their completion.
+	 *
+	 * @param threads threads to use
+	 * @throws Exception exceptions that are thrown from the threads
+	 */
+	protected void checkedThreadSimpleTest(CheckedThread[] threads)
+		throws Exception {
+
+		// start all threads
+		for (CheckedThread t: threads) {
+			t.start();
+		}
+
+		// wait for thread completion and check exceptions
+		for (CheckedThread t: threads) {
+			t.sync();
+		}
+	}
+
+	/**
+	 * Checked thread that calls {@link BlobServer#getStorageLocation(JobID, String)}
+	 */
+	public static class NameAddressableGetStorageLocation extends CheckedThread {
+		private final BlobServer server;
+		private final JobID jid;
+		private final String name;
+
+		public NameAddressableGetStorageLocation(BlobServer server, JobID jid, String name) {
+			this.server = server;
+			this.jid = jid;
+			this.name = name;
+		}
+
+		@Override
+		public void go() throws Exception {
+			server.getStorageLocation(jid, name);
+		}
+	}
+
+	/**
+	 * Tests concurrent calls to {@link BlobServer#getStorageLocation(JobID, String)}.
+	 */
+	@Test
+	public void testServerNameAddressableGetStorageLocationConcurrent() throws Exception {
+		BlobServer server = new BlobServer(new Configuration(), new VoidBlobStore());
+
+		try {
+			JobID jid = new JobID();
+			String stringKey = "my test key";
+			CheckedThread[] threads = new CheckedThread[] {
+				new NameAddressableGetStorageLocation(server, jid, stringKey),
+				new NameAddressableGetStorageLocation(server, jid, stringKey),
+				new NameAddressableGetStorageLocation(server, jid, stringKey)
+			};
+			checkedThreadSimpleTest(threads);
+		} finally {
+			server.close();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
 	@Test
 	public void testPutBufferSuccessful() throws IOException {
 		BlobServer server = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/28c3354e/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
index 98e6b3e..73f5819 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
@@ -43,8 +43,12 @@ import java.util.List;
 
 public class BlobLibraryCacheManagerTest {
 
+	/**
+	 * Tests that the {@link BlobLibraryCacheManager} cleans up after calling {@link
+	 * BlobLibraryCacheManager#unregisterJob(JobID)}.
+	 */
 	@Test
-	public void testLibraryCacheManagerCleanup() throws IOException, InterruptedException {
+	public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedException {
 
 		JobID jid = new JobID();
 		List<BlobKey> keys = new ArrayList<BlobKey>();
@@ -67,14 +71,9 @@ public class BlobLibraryCacheManagerTest {
 			libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval);
 			libraryCacheManager.registerJob(jid, keys, Collections.<URL>emptyList());
 
-			List<File> files = new ArrayList<File>();
-
-			for (BlobKey key : keys) {
-				files.add(libraryCacheManager.getFile(key));
-			}
-
-			assertEquals(2, files.size());
-			files.clear();
+			assertEquals(2, checkFilesExist(keys, libraryCacheManager, true));
+			assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries());
+			assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid));
 
 			libraryCacheManager.unregisterJob(jid);
 
@@ -86,25 +85,137 @@ public class BlobLibraryCacheManagerTest {
 					Thread.sleep(500);
 				}
 				while (libraryCacheManager.getNumberOfCachedLibraries() > 0 &&
-						System.currentTimeMillis() < deadline);
+					System.currentTimeMillis() < deadline);
 			}
 
 			// this fails if we exited via a timeout
 			assertEquals(0, libraryCacheManager.getNumberOfCachedLibraries());
+			assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid));
 
-			int caughtExceptions = 0;
+			// the blob cache should no longer contain the files
+			assertEquals(0, checkFilesExist(keys, libraryCacheManager, false));
 
-			for (BlobKey key : keys) {
-				// the blob cache should no longer contain the files
+			try {
+				bc.get(jid, "test");
+				fail("name-addressable BLOB should have been deleted");
+			} catch (IOException e) {
+				// expected
+			}
+
+			bc.close();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			if (server != null) {
+				server.close();
+			}
+
+			if (libraryCacheManager != null) {
 				try {
-					files.add(libraryCacheManager.getFile(key));
+					libraryCacheManager.shutdown();
 				}
-				catch (IOException ioe) {
-					caughtExceptions++;
+				catch (IOException e) {
+					e.printStackTrace();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Checks how many of the files given by blob keys are accessible.
+	 *
+	 * @param keys
+	 * 		blob keys to check
+	 * @param libraryCacheManager
+	 * 		cache manager to use
+	 * @param doThrow
+	 * 		whether exceptions should be ignored (<tt>false</tt>), or throws (<tt>true</tt>)
+	 *
+	 * @return number of files we were able to retrieve via {@link BlobLibraryCacheManager#getFile(BlobKey)}
+	 */
+	private int checkFilesExist(
+			List<BlobKey> keys, BlobLibraryCacheManager libraryCacheManager, boolean doThrow)
+			throws IOException {
+		int numFiles = 0;
+
+		for (BlobKey key : keys) {
+			try {
+				libraryCacheManager.getFile(key);
+				++numFiles;
+			} catch (IOException e) {
+				if (doThrow) {
+					throw e;
+				}
+			}
+		}
+
+		return numFiles;
+	}
+
+	/**
+	 * Tests that the {@link BlobLibraryCacheManager} cleans up after calling {@link
+	 * BlobLibraryCacheManager#unregisterTask(JobID, ExecutionAttemptID)}.
+	 */
+	@Test
+	public void testLibraryCacheManagerTaskCleanup() throws IOException, InterruptedException {
+
+		JobID jid = new JobID();
+		ExecutionAttemptID executionId1 = new ExecutionAttemptID();
+		ExecutionAttemptID executionId2 = new ExecutionAttemptID();
+		List<BlobKey> keys = new ArrayList<BlobKey>();
+		BlobServer server = null;
+		BlobLibraryCacheManager libraryCacheManager = null;
+
+		final byte[] buf = new byte[128];
+
+		try {
+			Configuration config = new Configuration();
+			server = new BlobServer(config, new VoidBlobStore());
+			InetSocketAddress blobSocketAddress = new InetSocketAddress(server.getPort());
+			BlobClient bc = new BlobClient(blobSocketAddress, config);
+
+			keys.add(bc.put(buf));
+			buf[0] += 1;
+			keys.add(bc.put(buf));
+			bc.put(jid, "test", buf);
+
+			long cleanupInterval = 1000l;
+			libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval);
+			libraryCacheManager.registerTask(jid, executionId1, keys, Collections.<URL>emptyList());
+			libraryCacheManager.registerTask(jid, executionId2, keys, Collections.<URL>emptyList());
+
+			assertEquals(2, checkFilesExist(keys, libraryCacheManager, true));
+			assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries());
+			assertEquals(2, libraryCacheManager.getNumberOfReferenceHolders(jid));
+
+			libraryCacheManager.unregisterTask(jid, executionId1);
+
+			assertEquals(2, checkFilesExist(keys, libraryCacheManager, true));
+			assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries());
+			assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid));
+
+			libraryCacheManager.unregisterTask(jid, executionId2);
+
+			// because we cannot guarantee that there are not thread races in the build system, we
+			// loop for a certain while until the references disappear
+			{
+				long deadline = System.currentTimeMillis() + 30000;
+				do {
+					Thread.sleep(100);
 				}
+				while (libraryCacheManager.getNumberOfCachedLibraries() > 0 &&
+						System.currentTimeMillis() < deadline);
 			}
 
-			assertEquals(2, caughtExceptions);
+			// this fails if we exited via a timeout
+			assertEquals(0, libraryCacheManager.getNumberOfCachedLibraries());
+			assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid));
+
+			// the blob cache should no longer contain the files
+			assertEquals(0, checkFilesExist(keys, libraryCacheManager, false));
 
 			bc.close();
 		} finally {


[2/9] flink git commit: [hotfix] Fix typos in Task and JobManager

Posted by ch...@apache.org.
[hotfix] Fix typos in Task and JobManager


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

Branch: refs/heads/master
Commit: 957cced552a60a9df21e76989f74d803eed6b924
Parents: 28c3354
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Wed Apr 19 15:39:03 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 14:57:35 2017 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/flink/runtime/taskmanager/Task.java   | 2 +-
 .../scala/org/apache/flink/runtime/jobmanager/JobManager.scala     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/957cced5/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 9dc6e34..65ce18c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -508,7 +508,7 @@ public class Task implements Runnable, TaskActions {
 	}
 
 	/**
-	 * The core work method that bootstraps the task and executes it code
+	 * The core work method that bootstraps the task and executes its code
 	 */
 	@Override
 	public void run() {

http://git-wip-us.apache.org/repos/asf/flink/blob/957cced5/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 100199b..9beae07 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -1792,7 +1792,7 @@ class JobManager(
       libraryCacheManager.unregisterJob(jobID)
     } catch {
       case t: Throwable =>
-        log.error(s"Could not properly unregister job $jobID form the library cache.", t)
+        log.error(s"Could not properly unregister job $jobID from the library cache.", t)
     }
     jobManagerMetricGroup.foreach(_.removeJob(jobID))
 


[3/9] flink git commit: [FLINK-6043] [web] Display exception timestamp

Posted by ch...@apache.org.
[FLINK-6043] [web] Display exception timestamp

This closes #3583.


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

Branch: refs/heads/master
Commit: 0f91083b2b497485b4375935ed7397583a03ca7c
Parents: 957cced
Author: zentol <ch...@apache.org>
Authored: Wed Mar 29 14:46:19 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 15:04:16 2017 +0200

----------------------------------------------------------------------
 .../handlers/JobExceptionsHandler.java          | 11 ++-
 .../handlers/JobExceptionsHandlerTest.java      |  5 +-
 .../utils/ArchivedExecutionGraphBuilder.java    |  7 +-
 .../utils/ArchivedJobGenerationUtils.java       |  3 +-
 .../app/partials/jobs/job.exceptions.jade       | 14 +++-
 .../executiongraph/AccessExecutionGraph.java    |  4 +-
 .../executiongraph/ArchivedExecutionGraph.java  |  6 +-
 .../flink/runtime/executiongraph/ErrorInfo.java | 85 ++++++++++++++++++++
 .../runtime/executiongraph/ExecutionGraph.java  | 56 +++++++++----
 .../ArchivedExecutionGraphTest.java             |  2 +-
 .../ExecutionGraphSuspendTest.java              |  2 +-
 .../query/AbstractQueryableStateITCase.java     |  2 +-
 12 files changed, 165 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
index 181b270..e31299b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.webmonitor.handlers;
 
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
 import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
@@ -78,9 +80,10 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
 		gen.writeStartObject();
 
 		// most important is the root failure cause
-		String rootException = graph.getFailureCauseAsString();
-		if (rootException != null && !rootException.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
-			gen.writeStringField("root-exception", rootException);
+		ErrorInfo rootException = graph.getFailureCause();
+		if (rootException != null && !rootException.getExceptionAsString().equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
+			gen.writeStringField("root-exception", rootException.getExceptionAsString());
+			gen.writeNumberField("timestamp", rootException.getTimestamp());
 		}
 
 		// we additionally collect all exceptions (up to a limit) that occurred in the individual tasks
@@ -105,6 +108,8 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
 				gen.writeStringField("exception", t);
 				gen.writeStringField("task", task.getTaskNameWithSubtaskIndex());
 				gen.writeStringField("location", locationString);
+				long timestamp = task.getStateTimestamp(ExecutionState.FAILED);
+				gen.writeNumberField("timestamp", timestamp == 0 ? -1 : timestamp);
 				gen.writeEndObject();
 				numExceptionsSoFar++;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
index f3df225..6016d01 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.webmonitor.handlers;
 
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
 import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
@@ -71,7 +72,8 @@ public class JobExceptionsHandlerTest {
 	private static void compareExceptions(AccessExecutionGraph originalJob, String json) throws IOException {
 		JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
 
-		Assert.assertEquals(originalJob.getFailureCauseAsString(), result.get("root-exception").asText());
+		Assert.assertEquals(originalJob.getFailureCause().getExceptionAsString(), result.get("root-exception").asText());
+		Assert.assertEquals(originalJob.getFailureCause().getTimestamp(), result.get("timestamp").asLong());
 
 		ArrayNode exceptions = (ArrayNode) result.get("all-exceptions");
 
@@ -81,6 +83,7 @@ public class JobExceptionsHandlerTest {
 				JsonNode exception = exceptions.get(x);
 
 				Assert.assertEquals(expectedSubtask.getFailureCauseAsString(), exception.get("exception").asText());
+				Assert.assertEquals(expectedSubtask.getStateTimestamp(ExecutionState.FAILED), exception.get("timestamp").asLong());
 				Assert.assertEquals(expectedSubtask.getTaskNameWithSubtaskIndex(), exception.get("task").asText());
 
 				TaskManagerLocation location = expectedSubtask.getCurrentAssignedResourceLocation();

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
index 935663c..57b300a 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedExecutionGraphBuilder.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
 import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.Preconditions;
@@ -47,7 +48,7 @@ public class ArchivedExecutionGraphBuilder {
 	private List<ArchivedExecutionJobVertex> verticesInCreationOrder;
 	private long[] stateTimestamps;
 	private JobStatus state;
-	private String failureCause;
+	private ErrorInfo failureCause;
 	private String jsonPlan;
 	private StringifiedAccumulatorResult[] archivedUserAccumulators;
 	private ArchivedExecutionConfig archivedExecutionConfig;
@@ -85,7 +86,7 @@ public class ArchivedExecutionGraphBuilder {
 		return this;
 	}
 
-	public ArchivedExecutionGraphBuilder setFailureCause(String failureCause) {
+	public ArchivedExecutionGraphBuilder setFailureCause(ErrorInfo failureCause) {
 		this.failureCause = failureCause;
 		return this;
 	}
@@ -126,7 +127,7 @@ public class ArchivedExecutionGraphBuilder {
 			verticesInCreationOrder != null ? verticesInCreationOrder : new ArrayList<>(tasks.values()),
 			stateTimestamps != null ? stateTimestamps : new long[JobStatus.values().length],
 			state != null ? state : JobStatus.FINISHED,
-			failureCause != null ? failureCause : "(null)",
+			failureCause,
 			jsonPlan != null ? jsonPlan : "{\"jobid\":\"" + jobID + "\", \"name\":\"" + jobName + "\", \"nodes\":[]}",
 			archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0],
 			serializedUserAccumulators != null ? serializedUserAccumulators : Collections.<String, SerializedValue<Object>>emptyMap(),

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
index dee04f2..3e4fc01 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/utils/ArchivedJobGenerationUtils.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.executiongraph.ArchivedExecution;
 import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ErrorInfo;
 import org.apache.flink.runtime.executiongraph.IOMetrics;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -132,7 +133,7 @@ public class ArchivedJobGenerationUtils {
 		originalJob = new ArchivedExecutionGraphBuilder()
 			.setJobID(new JobID())
 			.setTasks(tasks)
-			.setFailureCause("jobException")
+			.setFailureCause(new ErrorInfo(new Exception("jobException"), originalAttempt.getStateTimestamp(ExecutionState.FAILED)))
 			.setState(JobStatus.FINISHED)
 			.setStateTimestamps(new long[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
 			.setArchivedUserAccumulators(new StringifiedAccumulatorResult[]{acc1, acc2})

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime-web/web-dashboard/app/partials/jobs/job.exceptions.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.exceptions.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.exceptions.jade
index 1aa10b0..eee19f5 100644
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.exceptions.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.exceptions.jade
@@ -20,6 +20,12 @@
     .panel-title
       | Root exception
 
+  .panel-heading.clearfix
+    .panel-info.thin.last(ng-if="exceptions['timestamp'] != -1")
+      strong Timestamp:
+      = ' '
+      | {{ exceptions['timestamp'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+
   .panel-body
     pre.exception
       | {{ exceptions['root-exception'] }}
@@ -30,8 +36,14 @@
       | {{ exception.task }}
 
   .panel-heading.clearfix
+    .panel-info.thin(ng-if="exception.timestamp != -1")
+      strong Timestamp:
+      = ' '
+      | {{ exception.timestamp | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
     .panel-info.thin.last
-      span {{ exception.location }}
+      strong Location:
+      = ' '
+      | {{ exception.location }}
 
   .panel-body
     pre.exception

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java
index 3b064c3..df4ed2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java
@@ -66,9 +66,9 @@ public interface AccessExecutionGraph {
 	 * Returns the exception that caused the job to fail. This is the first root exception
 	 * that was not recoverable and triggered job failure.
 	 *
-	 * @return failure causing exception as a string, or {@code "(null)"}
+	 * @return failure causing exception, or null
 	 */
-	String getFailureCauseAsString();
+	ErrorInfo getFailureCause();
 
 	/**
 	 * Returns the job vertex for the given {@link JobVertexID}.

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
index b9db1e7..901b973 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
@@ -71,7 +71,7 @@ public class ArchivedExecutionGraph implements AccessExecutionGraph, Serializabl
 	 * The exception that caused the job to fail. This is set to the first root exception
 	 * that was not recoverable and triggered job failure
 	 */
-	private final String failureCause;
+	private final ErrorInfo failureCause;
 
 	// ------ Fields that are only relevant for archived execution graphs ------------
 	private final String jsonPlan;
@@ -93,7 +93,7 @@ public class ArchivedExecutionGraph implements AccessExecutionGraph, Serializabl
 			List<ArchivedExecutionJobVertex> verticesInCreationOrder,
 			long[] stateTimestamps,
 			JobStatus state,
-			String failureCause,
+			ErrorInfo failureCause,
 			String jsonPlan,
 			StringifiedAccumulatorResult[] archivedUserAccumulators,
 			Map<String, SerializedValue<Object>> serializedUserAccumulators,
@@ -141,7 +141,7 @@ public class ArchivedExecutionGraph implements AccessExecutionGraph, Serializabl
 	}
 
 	@Override
-	public String getFailureCauseAsString() {
+	public ErrorInfo getFailureCause() {
 		return failureCause;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ErrorInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ErrorInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ErrorInfo.java
new file mode 100644
index 0000000..d919bfa
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ErrorInfo.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph;
+
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+/**
+ * Simple container to hold an exception and the corresponding timestamp.
+ */
+public class ErrorInfo implements Serializable {
+
+	private static final long serialVersionUID = -6138942031953594202L;
+
+	private final transient Throwable exception;
+	private final long timestamp;
+
+	private volatile String exceptionAsString;
+
+	public ErrorInfo(Throwable exception, long timestamp) {
+		Preconditions.checkNotNull(exception);
+		Preconditions.checkArgument(timestamp > 0);
+
+		this.exception = exception;
+		this.timestamp = timestamp;
+	}
+
+	/**
+	 * Returns the contained exception.
+	 *
+	 * @return contained exception, or {@code "(null)"} if either no exception was set or this object has been deserialized
+	 */
+	Throwable getException() {
+		return exception;
+	}
+
+	/**
+	 * Returns the contained exception as a string.
+	 *
+	 * @return failure causing exception as a string, or {@code "(null)"}
+	 */
+	public String getExceptionAsString() {
+		if (exceptionAsString == null) {
+			exceptionAsString = ExceptionUtils.stringifyException(exception);
+		}
+		return exceptionAsString;
+	}
+
+	/**
+	 * Returns the timestamp for the contained exception.
+	 *
+	 * @return timestamp of contained exception, or 0 if no exception was set
+	 */
+	public long getTimestamp() {
+		return timestamp;
+	}
+
+	private void writeObject(ObjectOutputStream out) throws IOException {
+		// make sure that the exception was stringified so it isn't lost during serialization
+		if (exceptionAsString == null) {
+			exceptionAsString = ExceptionUtils.stringifyException(exception);
+		}
+		out.defaultWriteObject();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 7c13936..d4f7551 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -261,7 +261,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 	/** The exception that caused the job to fail. This is set to the first root exception
 	 * that was not recoverable and triggered job failure */
-	private volatile Throwable failureCause;
+	private volatile ErrorInfo failureCause;
 
 	// ------ Fields that are relevant to the execution and need to be cleared before archiving  -------
 
@@ -593,7 +593,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		return state;
 	}
 
-	public Throwable getFailureCause() {
+	public ErrorInfo getFailureCause() {
 		return failureCause;
 	}
 
@@ -610,11 +610,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	}
 
 	@Override
-	public String getFailureCauseAsString() {
-		return ExceptionUtils.stringifyException(failureCause);
-	}
-
-	@Override
 	public ExecutionJobVertex getJobVertex(JobVertexID id) {
 		return this.tasks.get(id);
 	}
@@ -1034,6 +1029,25 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	 * @param suspensionCause Cause of the suspension
 	 */
 	public void suspend(Throwable suspensionCause) {
+		suspend(new ErrorInfo(suspensionCause, System.currentTimeMillis()));
+	}
+
+	/**
+	 * Suspends the current ExecutionGraph.
+	 *
+	 * The JobStatus will be directly set to SUSPENDED iff the current state is not a terminal
+	 * state. All ExecutionJobVertices will be canceled and the postRunCleanup is executed.
+	 *
+	 * The SUSPENDED state is a local terminal state which stops the execution of the job but does
+	 * not remove the job from the HA job store so that it can be recovered by another JobManager.
+	 *
+	 * @param errorInfo ErrorInfo containing the cause of the suspension
+	 */
+	public void suspend(ErrorInfo errorInfo) {
+		Throwable suspensionCause = errorInfo != null
+			? errorInfo.getException()
+			: null;
+
 		while (true) {
 			JobStatus currentState = state;
 
@@ -1041,7 +1055,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 				// stay in a terminal state
 				return;
 			} else if (transitionState(currentState, JobStatus.SUSPENDED, suspensionCause)) {
-				this.failureCause = suspensionCause;
+				this.failureCause = errorInfo;
 
 				// make sure no concurrent local actions interfere with the cancellation
 				incrementGlobalModVersion();
@@ -1061,6 +1075,10 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		}
 	}
 
+	public void failGlobal(Throwable suspensionCause) {
+		failGlobal(new ErrorInfo(suspensionCause, System.currentTimeMillis()));
+	}
+
 	/**
 	 * Fails the execution graph globally. This failure will not be recovered by a specific
 	 * failover strategy, but results in a full restart of all tasks.
@@ -1070,9 +1088,13 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	 * exceptions that indicate a bug or an unexpected call race), and where a full restart is the
 	 * safe way to get consistency back.
 	 * 
-	 * @param t The exception that caused the failure.
+	 * @param errorInfo ErrorInfo containing the exception that caused the failure.
 	 */
-	public void failGlobal(Throwable t) {
+	public void failGlobal(ErrorInfo errorInfo) {
+		Throwable t = errorInfo != null
+			? errorInfo.getException()
+			: null;
+
 		while (true) {
 			JobStatus current = state;
 			// stay in these states
@@ -1082,14 +1104,14 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 				return;
 			}
 			else if (current == JobStatus.RESTARTING) {
-				this.failureCause = t;
+				this.failureCause = errorInfo;
 
 				if (tryRestartOrFail()) {
 					return;
 				}
 			}
-			else if (transitionState(current, JobStatus.FAILING, t)) {
-				this.failureCause = t;
+			else if (transitionState(current, JobStatus.FAILING,t)) {
+				this.failureCause = errorInfo;
 
 				// make sure no concurrent local actions interfere with the cancellation
 				incrementGlobalModVersion();
@@ -1375,6 +1397,9 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		JobStatus currentState = state;
 
 		if (currentState == JobStatus.FAILING || currentState == JobStatus.RESTARTING) {
+			Throwable failureCause = this.failureCause != null
+				? this.failureCause.getException()
+				: null;
 			synchronized (progressLock) {
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID(), failureCause);
@@ -1633,6 +1658,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		// see what this means for us. currently, the first FAILED state means -> FAILED
 		if (newExecutionState == ExecutionState.FAILED) {
 			final Throwable ex = error != null ? error : new FlinkException("Unknown Error (missing cause)");
+			long timestamp = execution.getStateTimestamp(ExecutionState.FAILED);
 
 			// by filtering out late failure calls, we can save some work in
 			// avoiding redundant local failover
@@ -1643,7 +1669,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 				catch (Throwable t) {
 					// bug in the failover strategy - fall back to global failover
 					LOG.warn("Error in failover strategy - falling back to global restart", t);
-					failGlobal(ex);
+					failGlobal(new ErrorInfo(ex, timestamp));
 				}
 			}
 		}
@@ -1674,7 +1700,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			archivedVerticesInCreationOrder,
 			stateTimestamps,
 			getState(),
-			getFailureCauseAsString(),
+			failureCause,
 			getJsonPlan(),
 			getAccumulatorResultsStringified(),
 			serializedUserAccumulators,

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
index 4e1d0f7..5ffc4ae 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
@@ -167,7 +167,7 @@ public class ArchivedExecutionGraphTest {
 		assertEquals(runtimeGraph.getJobID(), archivedGraph.getJobID());
 		assertEquals(runtimeGraph.getJobName(), archivedGraph.getJobName());
 		assertEquals(runtimeGraph.getState(), archivedGraph.getState());
-		assertEquals(runtimeGraph.getFailureCauseAsString(), archivedGraph.getFailureCauseAsString());
+		assertEquals(runtimeGraph.getFailureCause().getExceptionAsString(), archivedGraph.getFailureCause().getExceptionAsString());
 		assertEquals(runtimeGraph.getStatusTimestamp(JobStatus.CREATED), archivedGraph.getStatusTimestamp(JobStatus.CREATED));
 		assertEquals(runtimeGraph.getStatusTimestamp(JobStatus.RUNNING), archivedGraph.getStatusTimestamp(JobStatus.RUNNING));
 		assertEquals(runtimeGraph.getStatusTimestamp(JobStatus.FAILING), archivedGraph.getStatusTimestamp(JobStatus.FAILING));

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java
index b3af005..b3a8c33 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java
@@ -251,7 +251,7 @@ public class ExecutionGraphSuspendTest {
 
 		assertEquals(JobStatus.SUSPENDED, eg.getState());
 
-		assertEquals(exception, eg.getFailureCause());
+		assertEquals(exception, eg.getFailureCause().getException());
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/0f91083b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
index 6c8e758..f07113d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
@@ -365,7 +365,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							.mapTo(ClassTag$.MODULE$.<JobFound>apply(JobFound.class)),
 					deadline.timeLeft());
 
-			String failureCause = jobFound.executionGraph().getFailureCauseAsString();
+			String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString();
 
 			assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException"));
 			int causedByIndex = failureCause.indexOf("Caused by: ");


[8/9] flink git commit: [FLINK-7037] Remove scala suffic from flink-examples module

Posted by ch...@apache.org.
[FLINK-7037] Remove scala suffic from flink-examples module

This closes  #4221.


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

Branch: refs/heads/master
Commit: bd858845a4ade89f78c06f72444563adfa5a974b
Parents: cf1e57e
Author: zentol <ch...@apache.org>
Authored: Thu Jun 29 11:26:32 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 16:07:59 2017 +0200

----------------------------------------------------------------------
 flink-examples/flink-examples-batch/pom.xml     | 2 +-
 flink-examples/flink-examples-streaming/pom.xml | 2 +-
 flink-examples/flink-examples-table/pom.xml     | 2 +-
 flink-examples/pom.xml                          | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bd858845/flink-examples/flink-examples-batch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml
index cbff7e2..512fb1a 100644
--- a/flink-examples/flink-examples-batch/pom.xml
+++ b/flink-examples/flink-examples-batch/pom.xml
@@ -23,7 +23,7 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_${scala.binary.version}</artifactId>
+		<artifactId>flink-examples</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>

http://git-wip-us.apache.org/repos/asf/flink/blob/bd858845/flink-examples/flink-examples-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index 3c5c971..d6ff156 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -24,7 +24,7 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_${scala.binary.version}</artifactId>
+		<artifactId>flink-examples</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>

http://git-wip-us.apache.org/repos/asf/flink/blob/bd858845/flink-examples/flink-examples-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml
index 8574650..ecd874a 100644
--- a/flink-examples/flink-examples-table/pom.xml
+++ b/flink-examples/flink-examples-table/pom.xml
@@ -24,7 +24,7 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_${scala.binary.version}</artifactId>
+		<artifactId>flink-examples</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>

http://git-wip-us.apache.org/repos/asf/flink/blob/bd858845/flink-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml
index 38feb3d..76bfbc1 100644
--- a/flink-examples/pom.xml
+++ b/flink-examples/pom.xml
@@ -28,7 +28,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-examples_${scala.binary.version}</artifactId>
+	<artifactId>flink-examples</artifactId>
 	<name>flink-examples</name>
 	<packaging>pom</packaging>
 


[9/9] flink git commit: [FLINK-6823] Activate checkstyle for runtime/broadcast

Posted by ch...@apache.org.
[FLINK-6823] Activate checkstyle for runtime/broadcast

This closes #4068.


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

Branch: refs/heads/master
Commit: 4d18afed80774c57766752f8a3cc45133a4a457a
Parents: bd85884
Author: zentol <ch...@apache.org>
Authored: Fri Jun 2 22:18:49 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 16:08:03 2017 +0200

----------------------------------------------------------------------
 flink-runtime/pom.xml                           |  1 -
 .../runtime/broadcast/BroadcastVariableKey.java | 24 +++--
 .../broadcast/BroadcastVariableManager.java     | 58 ++++++------
 .../BroadcastVariableMaterialization.java       | 97 ++++++++++----------
 .../DefaultBroadcastVariableInitializer.java    | 17 ++--
 .../InitializationTypeConflictException.java    | 13 ++-
 .../MaterializationExpiredException.java        |  4 +
 7 files changed, 119 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index a21f753..15b502e 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -428,7 +428,6 @@ under the License.
 					<excludes>
 						**/runtime/akka/**,
 						**/runtime/blob/**,
-						**/runtime/broadcast/**,
 						**/runtime/checkpoint/**,
 						**/runtime/client/**,
 						**/runtime/clusterframework/**,       

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableKey.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableKey.java
index cfa87ae..372519e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableKey.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableKey.java
@@ -20,47 +20,51 @@ package org.apache.flink.runtime.broadcast;
 
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
+/**
+ * An identifier for a {@link BroadcastVariableMaterialization} based on the task's {@link JobVertexID}, broadcast
+ * variable name and iteration superstep.
+ */
 public class BroadcastVariableKey {
 
 	private final JobVertexID vertexId;
-	
+
 	private final String name;
-	
+
 	private final int superstep;
 
 	public BroadcastVariableKey(JobVertexID vertexId, String name, int superstep) {
 		if (vertexId == null || name == null || superstep <= 0) {
 			throw new IllegalArgumentException();
 		}
-		
+
 		this.vertexId = vertexId;
 		this.name = name;
 		this.superstep = superstep;
 	}
 
 	// ---------------------------------------------------------------------------------------------
-	
+
 	public JobVertexID getVertexId() {
 		return vertexId;
 	}
-	
+
 	public String getName() {
 		return name;
 	}
-	
+
 	public int getSuperstep() {
 		return superstep;
 	}
-	
+
 	// ---------------------------------------------------------------------------------------------
-	
+
 	@Override
 	public int hashCode() {
 		return 31 * superstep +
 				47 * name.hashCode() +
 				83 * vertexId.hashCode();
 	}
-	
+
 	@Override
 	public boolean equals(Object obj) {
 		if (obj != null && obj.getClass() == BroadcastVariableKey.class) {
@@ -73,7 +77,7 @@ public class BroadcastVariableKey {
 			return false;
 		}
 	}
-	
+
 	@Override
 	public String toString() {
 		return vertexId + " \"" + name + "\" (" + superstep + ')';

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
index 7d0454e..0cee70e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
@@ -18,34 +18,42 @@
 
 package org.apache.flink.runtime.broadcast;
 
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.runtime.io.network.api.reader.MutableReader;
 import org.apache.flink.runtime.operators.BatchTask;
 
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The BroadcastVariableManager is used to manage the materialization of broadcast variables. References to materialized
+ * broadcast variables are cached and shared between parallel subtasks. A reference count is maintained to track whether
+ * the materialization may be cleaned up.
+ */
 public class BroadcastVariableManager {
-	
+
 	private final ConcurrentHashMap<BroadcastVariableKey, BroadcastVariableMaterialization<?, ?>> variables =
 							new ConcurrentHashMap<BroadcastVariableKey, BroadcastVariableMaterialization<?, ?>>(16);
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * Materializes the broadcast variable for the given name, scoped to the given task and its iteration superstep. An
+	 * existing materialization created by another parallel subtask may be returned, if it hasn't expired yet.
+	 */
 	public <T> BroadcastVariableMaterialization<T, ?> materializeBroadcastVariable(String name, int superstep, BatchTask<?, ?> holder,
-			MutableReader<?> reader, TypeSerializerFactory<T> serializerFactory) throws IOException
-	{
+			MutableReader<?> reader, TypeSerializerFactory<T> serializerFactory) throws IOException {
 		final BroadcastVariableKey key = new BroadcastVariableKey(holder.getEnvironment().getJobVertexId(), name, superstep);
-		
+
 		while (true) {
 			final BroadcastVariableMaterialization<T, Object> newMat = new BroadcastVariableMaterialization<T, Object>(key);
-			
+
 			final BroadcastVariableMaterialization<?, ?> previous = variables.putIfAbsent(key, newMat);
-			
+
 			@SuppressWarnings("unchecked")
 			final BroadcastVariableMaterialization<T, ?> materialization = (previous == null) ? newMat : (BroadcastVariableMaterialization<T, ?>) previous;
-			
+
 			try {
 				materialization.materializeVariable(reader, serializerFactory, holder);
 				return materialization;
@@ -54,13 +62,13 @@ public class BroadcastVariableManager {
 				// concurrent release. as an optimization, try to replace the previous one with our version. otherwise we might spin for a while
 				// until the releaser removes the variable
 				// NOTE: This would also catch a bug prevented an expired materialization from ever being removed, so it acts as a future safeguard
-				
+
 				boolean replaceSuccessful = false;
 				try {
 					replaceSuccessful = variables.replace(key, materialization, newMat);
 				}
 				catch (Throwable t) {}
-				
+
 				if (replaceSuccessful) {
 					try {
 						newMat.materializeVariable(reader, serializerFactory, holder);
@@ -75,29 +83,27 @@ public class BroadcastVariableManager {
 			}
 		}
 	}
-	
-	
+
 	public void releaseReference(String name, int superstep, BatchTask<?, ?> referenceHolder) {
 		BroadcastVariableKey key = new BroadcastVariableKey(referenceHolder.getEnvironment().getJobVertexId(), name, superstep);
 		releaseReference(key, referenceHolder);
 	}
-	
+
 	public void releaseReference(BroadcastVariableKey key, BatchTask<?, ?> referenceHolder) {
 		BroadcastVariableMaterialization<?, ?> mat = variables.get(key);
-		
+
 		// release this reference
 		if (mat.decrementReference(referenceHolder)) {
 			// remove if no one holds a reference and no one concurrently replaced the entry
 			variables.remove(key, mat);
 		}
 	}
-	
-	
+
 	public void releaseAllReferencesFromTask(BatchTask<?, ?> referenceHolder) {
-		// go through all registered variables 
+		// go through all registered variables
 		for (Map.Entry<BroadcastVariableKey, BroadcastVariableMaterialization<?, ?>> entry : variables.entrySet()) {
 			BroadcastVariableMaterialization<?, ?> mat = entry.getValue();
-			
+
 			// release the reference
 			if (mat.decrementReferenceIfHeld(referenceHolder)) {
 				// remove if no one holds a reference and no one concurrently replaced the entry
@@ -105,10 +111,10 @@ public class BroadcastVariableManager {
 			}
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public int getNumberOfVariablesWithReferences() {
 		return this.variables.size();
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
index cea32e5..9c2c109 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.runtime.broadcast;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
@@ -32,54 +26,61 @@ import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.util.ReaderIterator;
 import org.apache.flink.runtime.plugable.DeserializationDelegate;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 /**
+ * This class represents a single materialization of a broadcast variable and maintains a reference count for it. If the
+ * reference count reaches zero the variable is no longer accessible and will eventually be garbage-collected.
+ *
  * @param <T> The type of the elements in the broadcasted data set.
  */
 public class BroadcastVariableMaterialization<T, C> {
-	
+
 	private static final Logger LOG = LoggerFactory.getLogger(BroadcastVariableMaterialization.class);
-	
-	
-	private final Set<BatchTask<?, ?>> references = new HashSet<BatchTask<?,?>>();
-	
+
+	private final Set<BatchTask<?, ?>> references = new HashSet<BatchTask<?, ?>>();
+
 	private final Object materializationMonitor = new Object();
-	
+
 	private final BroadcastVariableKey key;
-	
+
 	private ArrayList<T> data;
-	
+
 	private C transformed;
-	
+
 	private boolean materialized;
-	
+
 	private boolean disposed;
-	
-	
+
 	public BroadcastVariableMaterialization(BroadcastVariableKey key) {
 		this.key = key;
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	public void materializeVariable(MutableReader<?> reader, TypeSerializerFactory<?> serializerFactory, BatchTask<?, ?> referenceHolder)
-			throws MaterializationExpiredException, IOException
-	{
+			throws MaterializationExpiredException, IOException {
 		Preconditions.checkNotNull(reader);
 		Preconditions.checkNotNull(serializerFactory);
 		Preconditions.checkNotNull(referenceHolder);
-		
+
 		final boolean materializer;
-		
+
 		// hold the reference lock only while we track references and decide who should be the materializer
 		// that way, other tasks can de-register (in case of failure) while materialization is happening
 		synchronized (references) {
 			if (disposed) {
 				throw new MaterializationExpiredException();
 			}
-			
+
 			// sanity check
 			if (!references.add(referenceHolder)) {
 				throw new IllegalStateException(
@@ -87,64 +88,65 @@ public class BroadcastVariableMaterialization<T, C> {
 								referenceHolder.getEnvironment().getTaskInfo().getTaskNameWithSubtasks(),
 								key.toString()));
 			}
-			
+
 			materializer = references.size() == 1;
 		}
 
 		try {
 			@SuppressWarnings("unchecked")
 			final MutableReader<DeserializationDelegate<T>> typedReader = (MutableReader<DeserializationDelegate<T>>) reader;
-			
+
 			@SuppressWarnings("unchecked")
 			final TypeSerializer<T> serializer = ((TypeSerializerFactory<T>) serializerFactory).getSerializer();
 
 			final ReaderIterator<T> readerIterator = new ReaderIterator<T>(typedReader, serializer);
-			
+
 			if (materializer) {
 				// first one, so we need to materialize;
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Getting Broadcast Variable (" + key + ") - First access, materializing.");
 				}
-				
+
 				ArrayList<T> data = new ArrayList<T>();
-				
+
 				T element;
 				while ((element = readerIterator.next()) != null) {
 					data.add(element);
 				}
-				
+
 				synchronized (materializationMonitor) {
 					this.data = data;
 					this.materialized = true;
 					materializationMonitor.notifyAll();
 				}
-				
+
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Materialization of Broadcast Variable (" + key + ") finished.");
 				}
 			}
 			else {
 				// successor: discard all data and refer to the shared variable
-				
+
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Getting Broadcast Variable (" + key + ") - shared access.");
 				}
-				
+
 				T element = serializer.createInstance();
-				while ((element = readerIterator.next(element)) != null);
-				
+				while ((element = readerIterator.next(element)) != null) {
+				}
+
 				synchronized (materializationMonitor) {
 					while (!this.materialized && !disposed) {
 						materializationMonitor.wait();
 					}
 				}
-				
+
 			}
 		}
 		catch (Throwable t) {
 			// in case of an exception, we need to clean up big time
 			decrementReferenceIfHeld(referenceHolder);
-			
+
 			if (t instanceof IOException) {
 				throw (IOException) t;
 			} else {
@@ -152,15 +154,15 @@ public class BroadcastVariableMaterialization<T, C> {
 			}
 		}
 	}
-	
+
 	public boolean decrementReference(BatchTask<?, ?> referenceHolder) {
 		return decrementReferenceInternal(referenceHolder, true);
 	}
-	
+
 	public boolean decrementReferenceIfHeld(BatchTask<?, ?> referenceHolder) {
 		return decrementReferenceInternal(referenceHolder, false);
 	}
-	
+
 	private boolean decrementReferenceInternal(BatchTask<?, ?> referenceHolder, boolean errorIfNoReference) {
 		synchronized (references) {
 			if (disposed || references.isEmpty()) {
@@ -170,7 +172,7 @@ public class BroadcastVariableMaterialization<T, C> {
 					return false;
 				}
 			}
-			
+
 			if (!references.remove(referenceHolder)) {
 				if (errorIfNoReference) {
 					throw new IllegalStateException(
@@ -181,8 +183,7 @@ public class BroadcastVariableMaterialization<T, C> {
 					return false;
 				}
 			}
-			
-			
+
 			if (references.isEmpty()) {
 				disposed = true;
 				data = null;
@@ -193,9 +194,9 @@ public class BroadcastVariableMaterialization<T, C> {
 			}
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public List<T> getVariable() throws InitializationTypeConflictException {
 		if (!materialized) {
 			throw new IllegalStateException("The Broadcast Variable has not yet been materialized.");
@@ -203,7 +204,7 @@ public class BroadcastVariableMaterialization<T, C> {
 		if (disposed) {
 			throw new IllegalStateException("The Broadcast Variable has been disposed");
 		}
-		
+
 		synchronized (references) {
 			if (transformed != null) {
 				if (transformed instanceof List) {
@@ -219,7 +220,7 @@ public class BroadcastVariableMaterialization<T, C> {
 			}
 		}
 	}
-	
+
 	public C getVariable(BroadcastVariableInitializer<T, C> initializer) {
 		if (!materialized) {
 			throw new IllegalStateException("The Broadcast Variable has not yet been materialized.");
@@ -227,7 +228,7 @@ public class BroadcastVariableMaterialization<T, C> {
 		if (disposed) {
 			throw new IllegalStateException("The Broadcast Variable has been disposed");
 		}
-		
+
 		synchronized (references) {
 			if (transformed == null) {
 				transformed = initializer.initializeBroadcastVariable(data);

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/DefaultBroadcastVariableInitializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/DefaultBroadcastVariableInitializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/DefaultBroadcastVariableInitializer.java
index f18c431..dfa8bcd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/DefaultBroadcastVariableInitializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/DefaultBroadcastVariableInitializer.java
@@ -18,31 +18,34 @@
 
 package org.apache.flink.runtime.broadcast;
 
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-
+/**
+ * The default {@link BroadcastVariableInitializer} implementation that initializes the broadcast variable into a list.
+ */
 public class DefaultBroadcastVariableInitializer<T> implements BroadcastVariableInitializer<T, List<T>> {
 
 	@Override
 	public List<T> initializeBroadcastVariable(Iterable<T> data) {
 		ArrayList<T> list = new ArrayList<T>();
-		
+
 		for (T value : data) {
 			list.add(value);
 		}
 		return list;
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private static final DefaultBroadcastVariableInitializer<Object> INSTANCE = new DefaultBroadcastVariableInitializer<Object>();
-	
+
 	@SuppressWarnings("unchecked")
 	public static <E> DefaultBroadcastVariableInitializer<E> instance() {
 		return (DefaultBroadcastVariableInitializer<E>) INSTANCE;
 	}
-	
+
 	private DefaultBroadcastVariableInitializer() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/InitializationTypeConflictException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/InitializationTypeConflictException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/InitializationTypeConflictException.java
index f478210..b78f395 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/InitializationTypeConflictException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/InitializationTypeConflictException.java
@@ -18,16 +18,23 @@
 
 package org.apache.flink.runtime.broadcast;
 
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/**
+ * Indicates that a broadcast variable was initialized with a {@link DefaultBroadcastVariableInitializer} as a
+ * non-{@link java.util.List} type, and later accessed using {@link RuntimeContext#getBroadcastVariable(String)} which
+ * may only return lists.
+ */
 public class InitializationTypeConflictException extends Exception {
-	
+
 	private static final long serialVersionUID = -3930913982433642882L;
-	
+
 	private final Class<?> type;
 
 	public InitializationTypeConflictException(Class<?> type) {
 		this.type = type;
 	}
-	
+
 	public Class<?> getType() {
 		return type;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/4d18afed/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/MaterializationExpiredException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/MaterializationExpiredException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/MaterializationExpiredException.java
index 45f4f47..31b60f4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/MaterializationExpiredException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/MaterializationExpiredException.java
@@ -18,6 +18,10 @@
 
 package org.apache.flink.runtime.broadcast;
 
+/**
+ * Indicates that the {@link BroadcastVariableMaterialization} has materialized the broadcast variable at some point
+ * but discarded it already.
+ */
 public class MaterializationExpiredException extends Exception {
 	private static final long serialVersionUID = 7476456353634121934L;
 }


[5/9] flink git commit: Rebuild web-frontend

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf1e57ee/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 5fe4def..2f67d06 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,i){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,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"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 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)},"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 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+" 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,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.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,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,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,i.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,o,a;return o=function(t){return function(t){var r,n,o,a;return r=l.defer(),o=e.job.jid,a=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,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||s<i)&&(i=s);return n=!isNaN(i)&&i>u.noWatermark?i: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 i;return i=e.id,o(e).then(function(e){if(a[i]=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,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.state
 List(),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(fun
 ction(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.backPressureOperatorStats[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("Job
 PropertiesController",["$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,a,s){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=functio
 n(){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.SCHEDU
 LED,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,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace("&gt;",">")},i=function(r){var n,i,o;return d3.select(a).selectAll
 ("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.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,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(a).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,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,M,J,y;p=null,C=d3.behavior.zoom(),y=[],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()},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},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 i,o;return i="<div href='#/jobs/"+h+"/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):(
 g(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},s=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_wor
 kset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(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,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!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}),y[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,o,p,f),n.push(r.id),null!=r.inputs)for(h=r.inputs,l=0,d=h.leng
 th;l<d;l++)m=h[l],i(e,t,r,n,m);return e},M=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,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 y)i=y[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),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.sc
 ale(r).translate([o,a]),u.attr("transform","translate("+o+", "+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,i,o){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.toLowerCa
 se()){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+"joboverview").success(function(e){return function(t,n,i,o){return angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=
 e.setEndTimes(t)}}),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=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,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),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n<i;n++){if(o=t[n],o.id===e)return o;if(o.step_function&&(a=r(e,o.step_function)),a)return a}return null},t=i.defer(),l.job.promise.then(function(n){return function(i){var o;return o=r(e,a.plan.nodes),o.vertex=n.seekVertex(e),t.resolve(o)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,i;for(n=a.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 s;return s=i.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=i.defer(),l.job.promise.then(function(i){return function(i){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=i.defer(),l.job.promise.then(function(i){return function(i){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=i.defer(),l.job.promise.then(function(i){return function(i){return console.log(a.jid),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/accumulators").success(function(i){var o;return o=i["user-accumulators"],e.get(t.jobServer+"jobs/"+a.jid+"/ve
 rtices/"+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/"+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=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.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/"+a.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=functi
 on(r,n){var o;return o=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+a.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/"+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=i.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"> <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.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 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-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.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringi
 fy(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 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.setMetri
 cSize=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,a){if(o.id===r.id&&(i.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(i,o){var a;return n.s
 etupLSFor(i,o),a=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 a;if(a=n.metrics[i][o].indexOf(e.id),a===-1&&(a=_.findIndex(n.metrics[i][o],{id:e.id})),a===-1)return t.push(e)}),a.resolve(t)}),a.promise}}(this),this.getAllAvailableMetrics=function(n){return function(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 a,s;return a=t.defer(),s=o.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"/metrics?get="+s).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),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,i,o){var a;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.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){if(angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addC
 lass("fa-remove"),null!=e.error)return alert(e.error)})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r;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,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})},e.runJob=function(){var r;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,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:
 e.state.allowNonRestoredState}).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return i.go("single-job.plan.subtasks",{jobid:t.jobid})})},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.uploade
 r.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):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/"+encodeURIComponent(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(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)}),o.promise},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.loadOverview().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("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[0]}),o=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]})},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.taskmanagers)}),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("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,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"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 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)},"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 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+" 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,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("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,i,o,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,i.setupMetrics(r.jobid,t.vertices)}),c=function(t){var r,n,o,a;return o=function(t){return function(t){var r,n,o,a;return r=l.defer(),o=e.job.jid,a=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,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||s<i)&&(i=s);return n=!isNaN(i)&&i>u.noWatermark?i: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 i;return i=e.id,o(e).then(function(e){if(a[i]=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,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",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 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.backPressureOperatorStats[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,a,s){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{templ
 ate:"<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).relat
 iveTime(),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,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace("&gt;",">")},i=function(r){var n,i,o;return d3.select(a).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.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,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(a).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,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,M,J,y;p=null,C=d3.behavior.zoom(),y=[],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()},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},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 i,o;return i="<div href='#/jobs/"+h+"/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):(g(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},s=function(e,t,r,n,i,o){return nu
 ll==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(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,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!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}),y[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,o,p,f),n.push(r.id),null!=r.inputs)for(h=r.inputs,l=0,d=h.length;l<d;l++)m=h[l],i(e,t,r,n,m);return e},M=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,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 y)i=y[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),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([o,a]),u.attr("transform","translate("+o+", "+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,i,o){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=i.defer(),e.get(t.jobServer+"joboverview").success(function(e){return function(t,n,i,o){return angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=e.setEndTimes(t)}}),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=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,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),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n<i;n++){if(o=t[n],o.id===e)return o;if(o.step_function&&(a=r(e,o.
 step_function)),a)return a}return null},t=i.defer(),l.job.promise.then(function(n){return function(i){var o;return o=r(e,a.plan.nodes),o.vertex=n.seekVertex(e),t.resolve(o)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,i;for(n=a.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 s;return s=i.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=i.defer(),l.job.promise.then(function(i){return function(i){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=i.defer(),l.job.promise.then(function(i){return function(i){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=i.defer(),l.job.promise.then(function(i){return function(i){return console.log(a.jid),e.get(t.jobServer+"jobs/"+a.jid+"/vertices/"+r+"/accumulators").success(function(i){var o;return o=i["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: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/"+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=i.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+a.jid+"/checkpoints").success(function(e,t,n,i){re
 turn 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/"+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 o;return o=i.defer(),l.job.promise.then(function(i){return function(i){return e.get(t.jobServer+"jobs/"+a.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/"+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=i.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 b
 tn-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.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-ti
 tle").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-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.s
 etupLS(),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)},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 fu
 nction(o,a){if(o.id===r.id&&(i.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(i,o){var a;return n.setupLSFor(i,o),a=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 a;if(a=n.metrics[i][o].indexOf(e.id),a===-1&&(a=_.findIndex(n.metrics[i][o],{id:e.id})),a===-1)return t.push(e)}),a.resolve(t)}),a.promise}}(this),this.getAllAvailableMetrics=function(n){return function(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 a,s;return a=t.defer(),s=o.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+i+"
 /metrics?get="+s).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),a.resolve(r)}}(this)),a.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.loadOverview().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(){va
 r 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 a;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.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){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;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,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})},e.runJob=function(){var r;if("Submit"===e.state["submit-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit
 ting",e.state["plan-button"]="Show Plan",e.error=null,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:e.state.allowNonRestoredState}).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return i.go("single-job.plan.subtasks",{jobid:t.jobid})})},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):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.jo
 bServer+"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/"+encodeURIComponent(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(e)}),o.pro

<TRUNCATED>

[6/9] flink git commit: Rebuild web-frontend

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf1e57ee/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 0351c37..9ce4cc6 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,g,h,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],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()},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},g=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":g(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="<div href='#/jobs/"+h+"/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):(g(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,g,h,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),g=new dagreD3.render,c.select("g").call(g,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(h=r.inputs,l=0,d=h.length;l<d;l++)m=h[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+"joboverview").success(function(e){return function(t,n,i,o){return angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=e.setEndTimes(t)}}),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=f
 unction(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.de
 fer(),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.resolve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=i.defer(),l.job.promise.then(func
 tion(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+"/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=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.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,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.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 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-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 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 function(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.promis
 e}}(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("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.defaultState(),e.uploa
 der={},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;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,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){i
 f(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})},e.runJob=function(){var r;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,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:e.state.allowNonRestoredState}).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return i.go("single-job.plan.subtasks",{jobid:t.jobid})})},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):e.uploader.success="Uploaded!"},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelec
 tClass",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/"+encodeURIComponent(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(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)}),o.promise},this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",func
 tion(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.loadOverview().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("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(funct
 ion(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[0]}),o=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]})},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.taskmanagers)}),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("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.backPressureOperator
 Stats={},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.getMe
 trics(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.acc
 umulators=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.subt
 askAccumulators=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.backPressureOperatorStats[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)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}).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.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 bt
 n-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,g,h,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],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()},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},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"},s=function(e,t,r,n){var i,o;return i="<div href='#/jobs/"+h+"/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):(g(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.i
 d===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,g,h,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),g=new dagreD3.render,c.select("g").call(g,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(h=r.inputs,l=0,d=h.length;l<d;l++)m=h[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("z
 oom",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";ca
 se"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+"joboverview").success(function(e){return function(t,n,i,o){return angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=e.setEndTimes(t)}}),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.resolve(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.j
 obServer+"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+"/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=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").d
 irective("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"> <button type="button" ng-class="[btnClasses, {active: metric.view == \'chart\'}]" ng-click="setView(\'char
 t\')">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.c
 ontentGenerator(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,callb
 ack: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-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.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.valu
 es[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 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 function(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(){retur
 n 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.loadOverview().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.noa
 ccess=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.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){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;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,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})},e.runJob=function(){var r;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,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:e.state.allowNonRestoredState}).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return i.go("single-job.plan.subtasks",{jobid:t.jo
 bid})})},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=fun
 ction(){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/"+encodeURIComponent(n)+"/plan",{params:i}).success(function(e,t,r,n){return o.resolve(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)}),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[0]}),o=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]})},i["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(o)})}]).controller("SingleTaskManagerLogsController",["$scope","$s
 tateParams","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",f
 unction(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.taskmanagers)}),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}]),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=functi

<TRUNCATED>

[4/9] flink git commit: Rebuild web-frontend

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf1e57ee/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
index a5f6676..4b052a3 100644
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
@@ -21,6 +21,10 @@ limitations under the License.
   <div class="panel-heading clearfix">
     <div class="panel-title">Root exception</div>
   </div>
+  <div class="panel-heading clearfix">
+    <div ng-if="exceptions['timestamp'] != -1" class="panel-info thin last"><strong>Timestamp:</strong> {{ exceptions['timestamp'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+    </div>
+  </div>
   <div class="panel-body">
     <pre class="exception">{{ exceptions['root-exception'] }}</pre>
   </div>
@@ -30,7 +34,10 @@ limitations under the License.
     <div class="panel-title">{{ exception.task }}</div>
   </div>
   <div class="panel-heading clearfix">
-    <div class="panel-info thin last"><span>{{ exception.location }}</span></div>
+    <div ng-if="exception.timestamp != -1" class="panel-info thin"><strong>Timestamp:</strong> {{ exception.timestamp | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+    </div>
+    <div class="panel-info thin last"><strong>Location:</strong> {{ exception.location }}
+    </div>
   </div>
   <div class="panel-body">
     <pre class="exception">{{ exception.exception }}</pre>


[7/9] flink git commit: Rebuild web-frontend

Posted by ch...@apache.org.
Rebuild web-frontend


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

Branch: refs/heads/master
Commit: cf1e57eeb78068e54b16c4b45d61d4a0ab6e53b8
Parents: 0f91083
Author: zentol <ch...@apache.org>
Authored: Wed Jun 7 14:24:57 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Tue Jul 4 15:04:25 2017 +0200

----------------------------------------------------------------------
 flink-runtime-web/web-dashboard/web/js/hs/index.js          | 4 ++--
 flink-runtime-web/web-dashboard/web/js/index.js             | 4 ++--
 .../web-dashboard/web/partials/jobs/job.exceptions.html     | 9 ++++++++-
 3 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------