You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2015/10/20 09:58:52 UTC

[01/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Repository: flink
Updated Branches:
  refs/heads/master 5671c77c3 -> a6890b284


http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
new file mode 100644
index 0000000..a0c8312
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
@@ -0,0 +1,713 @@
+/*
+ * 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.test.recovery;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerProcess;
+import org.apache.flink.runtime.testutils.TaskManagerProcess;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.junit.AfterClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@Ignore
+public class ChaosMonkeyITCase {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ChaosMonkeyITCase.class);
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static File FileStateBackendBasePath;
+
+	private final static File CheckpointCompletedCoordination;
+
+	private final static File ProceedCoordination;
+
+	private final static String COMPLETED_PREFIX = "completed_";
+
+	private final static long LastElement = -1;
+
+	private final Random rand = new Random();
+
+	private int jobManagerPid;
+	private int taskManagerPid;
+
+	static {
+		try {
+			FileStateBackendBasePath = CommonTestUtils.createTempDirectory();
+			CheckpointCompletedCoordination = new File(FileStateBackendBasePath, COMPLETED_PREFIX);
+			ProceedCoordination = new File(FileStateBackendBasePath, "proceed");
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error in test setup. Could not create directory.", e);
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (ZooKeeper != null) {
+			ZooKeeper.shutdown();
+		}
+
+		if (FileStateBackendBasePath != null) {
+			FileUtils.deleteDirectory(FileStateBackendBasePath);
+		}
+	}
+
+	@Test
+	public void testChaosMonkey() throws Exception {
+		// Test config
+		final int numberOfJobManagers = 3;
+		final int numberOfTaskManagers = 3;
+		final int numberOfSlotsPerTaskManager = 2;
+
+		// The final count each source is counting to: 1...n
+		final int n = 5000;
+
+		// Parallelism for the program
+		final int parallelism = numberOfTaskManagers * numberOfSlotsPerTaskManager;
+
+		// The test should not run longer than this
+		final FiniteDuration testDuration = new FiniteDuration(10, TimeUnit.MINUTES);
+
+		// Every x seconds a random job or task manager is killed
+		//
+		// The job will will be running for $killEvery seconds and then a random Job/TaskManager
+		// will be killed. On recovery (which takes some time to bring up the new process etc.),
+		// this test will wait for task managers to reconnect before starting the next count down.
+		// Therefore the delay between retries is not important in this setup.
+		final FiniteDuration killEvery = new FiniteDuration(30, TimeUnit.SECONDS);
+
+		// Trigger a checkpoint every
+		final int checkpointingIntervalMs = 2000;
+
+		// Total number of kills
+		final int totalNumberOfKills = 5;
+
+		// -----------------------------------------------------------------------------------------
+
+		// Setup
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+				ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());
+
+		// Akka and restart timeouts
+		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms");
+		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s");
+		config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
+
+		if (checkpointingIntervalMs >= killEvery.toMillis()) {
+			throw new IllegalArgumentException("Relax! You want to kill processes every " +
+					killEvery + ", but the checkpointing interval is " +
+					checkpointingIntervalMs / 1000 + " seconds. Either decrease the interval or " +
+					"increase the kill interval. Otherwise, the program will not complete any " +
+					"checkpoint.");
+		}
+
+		// Task manager
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numberOfSlotsPerTaskManager);
+
+		ActorSystem testActorSystem = null;
+		LeaderRetrievalService leaderRetrievalService = null;
+		List<JobManagerProcess> jobManagerProcesses = new ArrayList<>();
+		List<TaskManagerProcess> taskManagerProcesses = new ArrayList<>();
+
+		try {
+			// Initial state
+			for (int i = 0; i < numberOfJobManagers; i++) {
+				jobManagerProcesses.add(createAndStartJobManagerProcess(config));
+			}
+
+			for (int i = 0; i < numberOfTaskManagers; i++) {
+				taskManagerProcesses.add(createAndStartTaskManagerProcess(config));
+			}
+
+			testActorSystem = AkkaUtils.createDefaultActorSystem();
+
+			// Leader listener
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			TestingListener leaderListener = new TestingListener();
+			leaderRetrievalService.start(leaderListener);
+
+			Deadline deadline = testDuration.fromNow();
+
+			// Wait for the new leader
+			int leaderIndex = waitForNewLeader(
+					leaderListener, jobManagerProcesses, deadline.timeLeft());
+
+			// Wait for the task managers to connect
+			waitForTaskManagers(
+					numberOfTaskManagers,
+					jobManagerProcesses.get(leaderIndex),
+					testActorSystem,
+					deadline.timeLeft());
+
+			// The job
+			JobGraph jobGraph = createJobGraph(n, CheckpointCompletedCoordination.getPath(),
+					ProceedCoordination.getPath(), parallelism, checkpointingIntervalMs);
+
+			LOG.info("Submitting job {}", jobGraph.getJobID());
+			submitJobGraph(jobGraph, jobManagerProcesses.get(leaderIndex), leaderListener,
+					testActorSystem, deadline.timeLeft());
+
+			LOG.info("Waiting for a checkpoint to complete before kicking off chaos");
+
+			// Wait for a checkpoint to complete
+			TestJvmProcess.waitForMarkerFiles(FileStateBackendBasePath, COMPLETED_PREFIX,
+					parallelism, deadline.timeLeft().toMillis());
+
+			LOG.info("Checkpoint completed... ready for chaos");
+
+			int currentKillNumber = 1;
+			int currentJobManagerKills = 0;
+			int currentTaskManagerKills = 0;
+
+			for (int i = 0; i < totalNumberOfKills; i++) {
+				LOG.info("Waiting for {} before next kill ({}/{})", killEvery, currentKillNumber++, totalNumberOfKills);
+				Thread.sleep(killEvery.toMillis());
+
+				LOG.info("Checking job status...");
+
+				JobStatus jobStatus = requestJobStatus(jobGraph.getJobID(),
+						jobManagerProcesses.get(leaderIndex), testActorSystem, deadline.timeLeft());
+
+				if (jobStatus != JobStatus.RUNNING && jobStatus != JobStatus.FINISHED) {
+					// Wait for it to run
+					LOG.info("Waiting for job status {}", JobStatus.RUNNING);
+					waitForJobRunning(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex),
+							testActorSystem, deadline.timeLeft());
+				}
+				else if (jobStatus == JobStatus.FINISHED) {
+					// Early finish
+					LOG.info("Job finished");
+					return;
+				}
+				else {
+					LOG.info("Job status is {}", jobStatus);
+				}
+
+				if (rand.nextBoolean()) {
+					LOG.info("Killing the leading JobManager");
+
+					JobManagerProcess newJobManager = createAndStartJobManagerProcess(config);
+
+					JobManagerProcess leader = jobManagerProcesses.remove(leaderIndex);
+					leader.destroy();
+					currentJobManagerKills++;
+
+					LOG.info("Killed {}", leader);
+
+					// Make sure to add the new job manager before looking for a new leader
+					jobManagerProcesses.add(newJobManager);
+
+					// Wait for the new leader
+					leaderIndex = waitForNewLeader(
+							leaderListener, jobManagerProcesses, deadline.timeLeft());
+
+					// Wait for the task managers to connect
+					waitForTaskManagers(
+							numberOfTaskManagers,
+							jobManagerProcesses.get(leaderIndex),
+							testActorSystem,
+							deadline.timeLeft());
+				}
+				else {
+					LOG.info("Killing a random TaskManager");
+					TaskManagerProcess newTaskManager = createAndStartTaskManagerProcess(config);
+
+					// Wait for this new task manager to be connected
+					waitForTaskManagers(
+							numberOfTaskManagers + 1,
+							jobManagerProcesses.get(leaderIndex),
+							testActorSystem,
+							deadline.timeLeft());
+
+					// Now it's safe to kill a process
+					int next = rand.nextInt(numberOfTaskManagers);
+					TaskManagerProcess taskManager = taskManagerProcesses.remove(next);
+
+					LOG.info("{} has been chosen. Killing process...", taskManager);
+
+					taskManager.destroy();
+					currentTaskManagerKills++;
+
+					// Add the new task manager after killing an old one
+					taskManagerProcesses.add(newTaskManager);
+				}
+			}
+
+			LOG.info("Chaos is over. Total kills: {} ({} job manager + {} task managers). " +
+							"Checking job status...",
+					totalNumberOfKills, currentJobManagerKills, currentTaskManagerKills);
+
+			// Signal the job to speed up (if it is not done yet)
+			TestJvmProcess.touchFile(ProceedCoordination);
+
+			// Wait for the job to finish
+			LOG.info("Waiting for job status {}", JobStatus.FINISHED);
+			waitForJobFinished(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex),
+					testActorSystem, deadline.timeLeft());
+
+			LOG.info("Job finished");
+
+			LOG.info("Waiting for job removal");
+			waitForJobRemoved(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex),
+					testActorSystem, deadline.timeLeft());
+			LOG.info("Job removed");
+
+			LOG.info("Checking clean recovery state...");
+			checkCleanRecoveryState(config);
+			LOG.info("Recovery state clean");
+		}
+		catch (Throwable t) {
+			System.out.println("#################################################");
+			System.out.println(" TASK MANAGERS");
+			System.out.println("#################################################");
+
+			for (TaskManagerProcess taskManagerProcess : taskManagerProcesses) {
+				taskManagerProcess.printProcessLog();
+			}
+
+			System.out.println("#################################################");
+			System.out.println(" JOB MANAGERS");
+			System.out.println("#################################################");
+
+			for (JobManagerProcess jobManagerProcess : jobManagerProcesses) {
+				jobManagerProcess.printProcessLog();
+			}
+
+			t.printStackTrace();
+		}
+		finally {
+			for (JobManagerProcess jobManagerProcess : jobManagerProcesses) {
+				if (jobManagerProcess != null) {
+					jobManagerProcess.destroy();
+				}
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			if (testActorSystem != null) {
+				testActorSystem.shutdown();
+			}
+		}
+	}
+
+	// - The test program --------------------------------------------------------------------------
+
+	private JobGraph createJobGraph(
+			int n,
+			String completedCheckpointMarker,
+			String proceedMarker,
+			int parallelism,
+			int checkpointingIntervalMs) {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(checkpointingIntervalMs);
+
+		int expectedResult = parallelism * n * (n + 1) / 2;
+
+		env.addSource(new CheckpointedSequenceSource(n, completedCheckpointMarker, proceedMarker))
+				.addSink(new CountingSink(parallelism, expectedResult))
+				.setParallelism(1);
+
+		return env.getStreamGraph().getJobGraph();
+	}
+
+	public static class CheckpointedSequenceSource extends RichParallelSourceFunction<Long>
+			implements Checkpointed<Long>, CheckpointNotifier {
+
+		private static final long serialVersionUID = 0L;
+
+		private final long end;
+
+		private final String completedCheckpointMarkerFilePath;
+
+		private final File proceedFile;
+
+		private long current = 0;
+
+		private volatile boolean isRunning = true;
+
+		public CheckpointedSequenceSource(long end, String completedCheckpointMarkerFilePath, String proceedMarkerFilePath) {
+			checkArgument(end >= 0, "Negative final count");
+
+			this.end = end;
+			this.completedCheckpointMarkerFilePath = completedCheckpointMarkerFilePath;
+			this.proceedFile = new File(proceedMarkerFilePath);
+		}
+
+		@Override
+		public void run(SourceContext<Long> ctx) throws Exception {
+			while (isRunning) {
+
+				if (!proceedFile.exists()) {
+					Thread.sleep(50);
+				}
+
+				synchronized (ctx.getCheckpointLock()) {
+					if (current <= end) {
+						ctx.collect(current++);
+					}
+					else {
+						ctx.collect(LastElement);
+						return;
+					}
+				}
+			}
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			LOG.info("Snapshotting state {} @ ID {}.", current, checkpointId);
+			return current;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			LOG.info("Restoring state {}/{}", state, end);
+			current = state;
+		}
+
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			LOG.info("Checkpoint {} completed.", checkpointId);
+
+			int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+			TestJvmProcess.touchFile(new File(completedCheckpointMarkerFilePath + taskIndex));
+		}
+	}
+
+	public static class CountingSink extends RichSinkFunction<Long>
+			implements Checkpointed<CountingSink>, CheckpointNotifier {
+
+		private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class);
+
+		private static final long serialVersionUID = 0L;
+
+		private final int parallelism;
+
+		private final long expectedFinalCount;
+
+		private long current;
+
+		private int numberOfReceivedLastElements;
+
+
+		public CountingSink(int parallelism, long expectedFinalCount) {
+			this.expectedFinalCount = expectedFinalCount;
+			this.parallelism = parallelism;
+		}
+
+		@Override
+		public void invoke(Long value) throws Exception {
+			if (value == LastElement) {
+				numberOfReceivedLastElements++;
+
+				if (numberOfReceivedLastElements == parallelism) {
+					if (current != expectedFinalCount) {
+						throw new Exception("Unexpected final result " + current);
+					}
+					else {
+						LOG.info("Final result " + current);
+					}
+				}
+				else if (numberOfReceivedLastElements > parallelism) {
+					throw new IllegalStateException("Received more elements than parallelism.");
+				}
+			}
+			else {
+				current += value;
+			}
+		}
+
+		@Override
+		public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			LOG.info("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId);
+			return this;
+		}
+
+		@Override
+		public void restoreState(CountingSink state) {
+			LOG.info("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements);
+			this.current = state.current;
+			this.numberOfReceivedLastElements = state.numberOfReceivedLastElements;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			LOG.info("Checkpoint {} completed.", checkpointId);
+		}
+	}
+
+	// - Utilities ---------------------------------------------------------------------------------
+
+	private void submitJobGraph(
+			JobGraph jobGraph,
+			JobManagerProcess jobManager,
+			TestingListener leaderListener,
+			ActorSystem actorSystem,
+			FiniteDuration timeout) throws Exception {
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		UUID jobManagerLeaderId = leaderListener.getLeaderSessionID();
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, jobManagerLeaderId);
+
+		jobManagerGateway.tell(new JobManagerMessages.SubmitJob(jobGraph, ListeningBehaviour.DETACHED));
+	}
+
+	private void checkCleanRecoveryState(Configuration config) throws Exception {
+		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+				ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
+		List<String> jobGraphs = ZooKeeper.getClient().getChildren()
+				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
+		assertEquals("Unclean job graphs: " + jobGraphs, 0, jobGraphs.size());
+
+		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
+		List<String> checkpoints = ZooKeeper.getClient().getChildren()
+				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
+		assertEquals("Unclean checkpoints: " + checkpoints, 0, checkpoints.size());
+
+		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
+		List<String> checkpointCounter = ZooKeeper.getClient().getChildren()
+				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
+		assertEquals("Unclean checkpoint counter: " + checkpointCounter, 0, checkpointCounter.size());
+
+		LOG.info("ZooKeeper state is clean");
+
+		LOG.info("Checking file system backend state...");
+
+		File fsCheckpoints = new File(config.getString(ConfigConstants.STATE_BACKEND_FS_DIR, ""));
+
+		LOG.info("Checking " + fsCheckpoints);
+
+		File[] files = fsCheckpoints.listFiles();
+		if (files == null) {
+			fail(fsCheckpoints + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles()));
+		}
+		else {
+			assertEquals("Unclean file system checkpoints: " + Arrays.toString(fsCheckpoints.listFiles()),
+					0, files.length);
+		}
+
+		File fsRecovery = new File(config.getString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, ""));
+
+		LOG.info("Checking " + fsRecovery);
+
+		files = fsRecovery.listFiles();
+		if (files == null) {
+			fail(fsRecovery + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles()));
+		}
+		else {
+			assertEquals("Unclean file system checkpoints: " + Arrays.toString(fsRecovery.listFiles()),
+					0, files.length);
+		}
+	}
+
+	private void waitForJobRemoved(
+			JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout)
+			throws Exception {
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null);
+
+		Future<Object> archiveFuture = jobManagerGateway.ask(JobManagerMessages.getRequestArchive(), timeout);
+
+		ActorRef archive = ((JobManagerMessages.ResponseArchive) Await.result(archiveFuture, timeout)).actor();
+
+		AkkaActorGateway archiveGateway = new AkkaActorGateway(archive, null);
+
+		Deadline deadline = timeout.fromNow();
+
+		while (deadline.hasTimeLeft()) {
+			JobManagerMessages.JobStatusResponse resp = JobManagerActorTestUtils
+					.requestJobStatus(jobId, archiveGateway, deadline.timeLeft());
+
+			if (resp instanceof JobManagerMessages.JobNotFound) {
+				Thread.sleep(100);
+			}
+			else {
+				return;
+			}
+		}
+	}
+
+	private JobStatus requestJobStatus(
+			JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout)
+			throws Exception {
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null);
+
+		JobManagerMessages.JobStatusResponse resp = JobManagerActorTestUtils
+				.requestJobStatus(jobId, jobManagerGateway, timeout);
+
+		if (resp instanceof JobManagerMessages.CurrentJobStatus) {
+			JobManagerMessages.CurrentJobStatus jobStatusResponse = (JobManagerMessages
+					.CurrentJobStatus) resp;
+
+			return jobStatusResponse.status();
+		}
+		else if (resp instanceof JobManagerMessages.JobNotFound) {
+			return JobStatus.RESTARTING;
+		}
+
+		throw new IllegalStateException("Unexpected response from JobManager");
+	}
+
+	private void waitForJobRunning(
+			JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout)
+			throws Exception {
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null);
+
+		JobManagerActorTestUtils.waitForJobStatus(jobId, JobStatus.RUNNING, jobManagerGateway, timeout);
+	}
+
+	private void waitForJobFinished(
+			JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout)
+			throws Exception {
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null);
+
+		JobManagerActorTestUtils.waitForJobStatus(jobId, JobStatus.FINISHED, jobManagerGateway, timeout);
+	}
+
+	private void waitForTaskManagers(
+			int minimumNumberOfTaskManagers,
+			JobManagerProcess jobManager,
+			ActorSystem actorSystem,
+			FiniteDuration timeout) throws Exception {
+
+		LOG.info("Waiting for {} task managers to connect to leading {}",
+				minimumNumberOfTaskManagers, jobManager);
+
+		ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout);
+		AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null);
+
+		JobManagerActorTestUtils.waitForTaskManagers(
+				minimumNumberOfTaskManagers, jobManagerGateway, timeout);
+
+		LOG.info("All task managers connected");
+	}
+
+	private int waitForNewLeader(
+			TestingListener leaderListener,
+			List<JobManagerProcess> jobManagerProcesses,
+			FiniteDuration timeout) throws Exception {
+
+		LOG.info("Waiting for new leader notification");
+		leaderListener.waitForNewLeader(timeout.toMillis());
+
+		LOG.info("Leader: {}:{}", leaderListener.getAddress(), leaderListener.getLeaderSessionID());
+
+		String currentLeader = leaderListener.getAddress();
+
+		int leaderIndex = -1;
+
+		for (int i = 0; i < jobManagerProcesses.size(); i++) {
+			JobManagerProcess jobManager = jobManagerProcesses.get(i);
+			if (jobManager.getJobManagerAkkaURL().equals(currentLeader)) {
+				leaderIndex = i;
+				break;
+			}
+		}
+
+		if (leaderIndex == -1) {
+			throw new IllegalStateException("Failed to determine which process is leader");
+		}
+
+		return leaderIndex;
+	}
+
+	private JobManagerProcess createAndStartJobManagerProcess(Configuration config)
+			throws Exception {
+
+		JobManagerProcess jobManager = new JobManagerProcess(jobManagerPid++, config);
+		jobManager.createAndStart();
+		LOG.info("Created and started {}.", jobManager);
+
+		return jobManager;
+	}
+
+	private TaskManagerProcess createAndStartTaskManagerProcess(Configuration config)
+			throws Exception {
+
+		TaskManagerProcess taskManager = new TaskManagerProcess(taskManagerPid++, config);
+		taskManager.createAndStart();
+		LOG.info("Created and started {}.", taskManager);
+
+		return taskManager;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
new file mode 100644
index 0000000..54ddf7e
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
@@ -0,0 +1,395 @@
+/*
+ * 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.test.recovery;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerProcess;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLongArray;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class JobManagerCheckpointRecoveryITCase {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	private static final File FileStateBackendBasePath;
+
+	static {
+		try {
+			FileStateBackendBasePath = CommonTestUtils.createTempDirectory();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error in test setup. Could not create directory.", e);
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		ZooKeeper.shutdown();
+
+		if (FileStateBackendBasePath != null) {
+			FileUtils.deleteDirectory(FileStateBackendBasePath);
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		if (FileStateBackendBasePath != null) {
+			FileUtils.cleanDirectory(FileStateBackendBasePath);
+		}
+
+		ZooKeeper.deleteAll();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	private static final int Parallelism = 8;
+
+	private static final CountDownLatch CompletedCheckpointsLatch = new CountDownLatch(2);
+
+	private static final AtomicLongArray RecoveredStates = new AtomicLongArray(Parallelism);
+
+	private static final CountDownLatch FinalCountLatch = new CountDownLatch(1);
+
+	private static final AtomicReference<Long> FinalCount = new AtomicReference<>();
+
+	private static final long LastElement = -1;
+
+	/**
+	 * Simple checkpointed streaming sum.
+	 *
+	 * <p>The sources (Parallelism) count until sequenceEnd. The sink (1) sums up all counts and
+	 * returns it to the main thread via a static variable. We wait until some checkpoints are
+	 * completed and sanity check that the sources recover with an updated state to make sure that
+	 * this test actually tests something.
+	 */
+	@Test
+	public void testCheckpointedStreamingSumProgram() throws Exception {
+		// Config
+		final int checkpointingInterval = 200;
+		final int sequenceEnd = 5000;
+		final long expectedSum = Parallelism * sequenceEnd * (sequenceEnd + 1) / 2;
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+		env.setParallelism(Parallelism);
+		env.enableCheckpointing(checkpointingInterval);
+
+		env
+				.addSource(new CheckpointedSequenceSource(sequenceEnd))
+				.addSink(new CountingSink())
+				.setParallelism(1);
+
+		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(ZooKeeper
+				.getConnectString(), FileStateBackendBasePath.getPath());
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Parallelism);
+
+		ActorSystem testSystem = null;
+		JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2];
+		LeaderRetrievalService leaderRetrievalService = null;
+		ActorSystem taskManagerSystem = null;
+
+		try {
+			final Deadline deadline = TestTimeOut.fromNow();
+
+			// Test actor system
+			testSystem = AkkaUtils.createActorSystem(new Configuration(),
+					new Some<>(new Tuple2<String, Object>("localhost", 0)));
+
+			// The job managers
+			jobManagerProcess[0] = new JobManagerProcess(0, config);
+			jobManagerProcess[1] = new JobManagerProcess(1, config);
+
+			jobManagerProcess[0].createAndStart();
+			jobManagerProcess[1].createAndStart();
+
+			// Leader listener
+			TestingListener leaderListener = new TestingListener();
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService.start(leaderListener);
+
+			// The task manager
+			taskManagerSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+			TaskManager.startTaskManagerComponentsAndActor(
+					config, taskManagerSystem, "localhost",
+					Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
+					false, StreamingMode.STREAMING, TaskManager.class);
+
+			{
+				// Initial submission
+				leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+				String leaderAddress = leaderListener.getAddress();
+				UUID leaderId = leaderListener.getLeaderSessionID();
+
+				// Get the leader ref
+				ActorRef leaderRef = AkkaUtils.getActorRef(
+						leaderAddress, testSystem, deadline.timeLeft());
+				ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId);
+
+				// Submit the job in detached mode
+				leader.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED));
+
+				JobManagerActorTestUtils.waitForJobStatus(
+						jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft());
+			}
+
+			// Who's the boss?
+			JobManagerProcess leadingJobManagerProcess;
+			if (jobManagerProcess[0].getJobManagerAkkaURL().equals(leaderListener.getAddress())) {
+				leadingJobManagerProcess = jobManagerProcess[0];
+			}
+			else {
+				leadingJobManagerProcess = jobManagerProcess[1];
+			}
+
+			CompletedCheckpointsLatch.await();
+
+			// Kill the leading job manager process
+			leadingJobManagerProcess.destroy();
+
+			{
+				// Recovery by the standby JobManager
+				leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+				String leaderAddress = leaderListener.getAddress();
+				UUID leaderId = leaderListener.getLeaderSessionID();
+
+				ActorRef leaderRef = AkkaUtils.getActorRef(
+						leaderAddress, testSystem, deadline.timeLeft());
+				ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId);
+
+				JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING,
+						leader, deadline.timeLeft());
+			}
+
+			// Wait to finish
+			FinalCountLatch.await();
+
+			assertEquals(expectedSum, (long) FinalCount.get());
+
+			for (int i = 0; i < Parallelism; i++) {
+				assertNotEquals(0, RecoveredStates.get(i));
+			}
+		}
+		catch (Throwable t) {
+			// In case of an error, print the job manager process logs.
+			if (jobManagerProcess[0] != null) {
+				jobManagerProcess[0].printProcessLog();
+			}
+
+			if (jobManagerProcess[1] != null) {
+				jobManagerProcess[1].printProcessLog();
+			}
+
+			t.printStackTrace();
+		}
+		finally {
+			if (jobManagerProcess[0] != null) {
+				jobManagerProcess[0].destroy();
+			}
+
+			if (jobManagerProcess[1] != null) {
+				jobManagerProcess[1].destroy();
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			if (taskManagerSystem != null) {
+				taskManagerSystem.shutdown();
+			}
+
+			if (testSystem != null) {
+				testSystem.shutdown();
+			}
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * A checkpointed source, which emits elements from 0 to a configured number.
+	 */
+	public static class CheckpointedSequenceSource extends RichParallelSourceFunction<Long>
+			implements Checkpointed<Long> {
+
+		private static final Logger LOG = LoggerFactory.getLogger(CheckpointedSequenceSource.class);
+
+		private static final long serialVersionUID = 0L;
+
+		private static final CountDownLatch sync = new CountDownLatch(Parallelism);
+
+		private final long end;
+
+		private long current = 0;
+
+		private volatile boolean isRunning = true;
+
+		public CheckpointedSequenceSource(long end) {
+			checkArgument(end >= 0, "Negative final count");
+			this.end = end;
+		}
+
+		@Override
+		public void run(SourceContext<Long> ctx) throws Exception {
+			while (isRunning) {
+				synchronized (ctx.getCheckpointLock()) {
+					if (current <= end) {
+						ctx.collect(current++);
+					}
+					else {
+						ctx.collect(LastElement);
+						return;
+					}
+				}
+
+				// Slow down until some checkpoints are completed
+				if (sync.getCount() != 0) {
+					Thread.sleep(100);
+				}
+			}
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			LOG.debug("Snapshotting state {} @ ID {}.", current, checkpointId);
+			return current;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			LOG.debug("Restoring state {}", state);
+
+			// This is necessary to make sure that something is recovered at all. Otherwise it
+			// might happen that the job is restarted from the beginning.
+			RecoveredStates.set(getRuntimeContext().getIndexOfThisSubtask(), state);
+
+			sync.countDown();
+
+			current = state;
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+	/**
+	 * A checkpointed sink, which sums up its input and notifies the main thread after all inputs
+	 * are exhausted.
+	 */
+	public static class CountingSink implements SinkFunction<Long>, Checkpointed<CountingSink>,
+			CheckpointNotifier {
+
+		private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class);
+
+		private static final long serialVersionUID = 1436484290453629091L;
+
+		private long current = 0;
+
+		private int numberOfReceivedLastElements;
+
+		@Override
+		public void invoke(Long value) throws Exception {
+			if (value == LastElement) {
+				numberOfReceivedLastElements++;
+
+				if (numberOfReceivedLastElements == Parallelism) {
+					FinalCount.set(current);
+					FinalCountLatch.countDown();
+				}
+				else if (numberOfReceivedLastElements > Parallelism) {
+					throw new IllegalStateException("Received more elements than parallelism.");
+				}
+			}
+			else {
+				current += value;
+			}
+		}
+
+		@Override
+		public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			LOG.debug("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId);
+			return this;
+		}
+
+		@Override
+		public void restoreState(CountingSink state) {
+			LOG.debug("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements);
+			this.current = state.current;
+			this.numberOfReceivedLastElements = state.numberOfReceivedLastElements;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			LOG.debug("Checkpoint {} completed.", checkpointId);
+			CompletedCheckpointsLatch.countDown();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java
new file mode 100644
index 0000000..66565dd
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.recovery;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test the recovery of a simple batch program in the case of JobManager process failure.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class JobManagerProcessFailureBatchRecoveryITCase extends AbstractJobManagerProcessFailureRecoveryITCase {
+
+	// --------------------------------------------------------------------------------------------
+	//  Parametrization (run pipelined and batch)
+	// --------------------------------------------------------------------------------------------
+
+	private final ExecutionMode executionMode;
+
+	public JobManagerProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) {
+		this.executionMode = executionMode;
+	}
+
+	@Parameterized.Parameters
+	public static Collection<Object[]> executionMode() {
+		return Arrays.asList(new Object[][]{
+				{ExecutionMode.PIPELINED},
+				{ExecutionMode.BATCH}});
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Test the program
+	// --------------------------------------------------------------------------------------------
+
+	// This is slightly modified copy the task manager process failure program.
+	@Override
+	public void testJobManagerFailure(String zkQuorum, final File coordinateDir) throws Exception {
+		Configuration config = new Configuration();
+		config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+		config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zkQuorum);
+
+		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"leader", 1, config);
+		env.setParallelism(PARALLELISM);
+		env.setNumberOfExecutionRetries(1);
+		env.getConfig().setExecutionMode(executionMode);
+		env.getConfig().disableSysoutLogging();
+
+		final long NUM_ELEMENTS = 100000L;
+		final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)
+				// make sure every mapper is involved (no one is skipped because of lazy split assignment)
+				.rebalance()
+				// the majority of the behavior is in the MapFunction
+				.map(new RichMapFunction<Long, Long>() {
+
+					private final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
+
+					private boolean markerCreated = false;
+					private boolean checkForProceedFile = true;
+
+					@Override
+					public Long map(Long value) throws Exception {
+						if (!markerCreated) {
+							int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+							AbstractTaskManagerProcessFailureRecoveryTest.touchFile(
+									new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex));
+							markerCreated = true;
+						}
+
+						// check if the proceed file exists
+						if (checkForProceedFile) {
+							if (proceedFile.exists()) {
+								checkForProceedFile = false;
+							}
+							else {
+								// otherwise wait so that we make slow progress
+								Thread.sleep(100);
+							}
+						}
+						return value;
+					}
+				})
+				.reduce(new ReduceFunction<Long>() {
+					@Override
+					public Long reduce(Long value1, Long value2) {
+						return value1 + value2;
+					}
+				})
+				// The check is done in the mapper, because the client can currently not handle
+				// job manager losses/reconnects.
+				.flatMap(new RichFlatMapFunction<Long, Long>() {
+					@Override
+					public void flatMap(Long value, Collector<Long> out) throws Exception {
+						assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, (long) value);
+
+						int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+						AbstractTaskManagerProcessFailureRecoveryTest.touchFile(
+								new File(coordinateDir, FINISH_MARKER_FILE_PREFIX + taskIndex));
+					}
+				});
+
+		result.output(new DiscardingOutputFormat<Long>());
+
+		env.execute();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
deleted file mode 100644
index f2b8c31..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.recovery;
-
-import org.apache.flink.api.common.ExecutionMode;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collection;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test the recovery of a simple batch program in the case of TaskManager process failure.
- */
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class ProcessFailureBatchRecoveryITCase extends AbstractProcessFailureRecoveryTest {
-
-	// --------------------------------------------------------------------------------------------
-	//  Parametrization (run pipelined and batch)
-	// --------------------------------------------------------------------------------------------
-
-	private final ExecutionMode executionMode;
-
-	public ProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) {
-		this.executionMode = executionMode;
-	}
-
-	@Parameterized.Parameters
-	public static Collection<Object[]> executionMode() {
-		return Arrays.asList(new Object[][]{
-				{ExecutionMode.PIPELINED},
-				{ExecutionMode.BATCH}});
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Test the program
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void testProgram(int jobManagerPort, final File coordinateDir) throws Exception {
-
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort);
-		env.setParallelism(PARALLELISM);
-		env.setNumberOfExecutionRetries(1);
-		env.getConfig().setExecutionMode(executionMode);
-		env.getConfig().disableSysoutLogging();
-
-		final long NUM_ELEMENTS = 100000L;
-		final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)
-
-				// make sure every mapper is involved (no one is skipped because of lazy split assignment)
-				.rebalance()
-						// the majority of the behavior is in the MapFunction
-				.map(new RichMapFunction<Long, Long>() {
-
-					private final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
-
-					private boolean markerCreated = false;
-					private boolean checkForProceedFile = true;
-
-					@Override
-					public Long map(Long value) throws Exception {
-						if (!markerCreated) {
-							int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
-							touchFile(new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex));
-							markerCreated = true;
-						}
-
-						// check if the proceed file exists
-						if (checkForProceedFile) {
-							if (proceedFile.exists()) {
-								checkForProceedFile = false;
-							} else {
-								// otherwise wait so that we make slow progress
-								Thread.sleep(100);
-							}
-						}
-						return value;
-					}
-				})
-				.reduce(new ReduceFunction<Long>() {
-					@Override
-					public Long reduce(Long value1, Long value2) {
-						return value1 + value2;
-					}
-				});
-
-		long sum = result.collect().get(0);
-		assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, sum);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
index 945a78c..6dce370 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
@@ -115,13 +115,13 @@ public class ProcessFailureCancelingITCase {
 					"-Dlog4j.configuration=file:" + tempLogFile.getAbsolutePath(),
 					"-Xms80m", "-Xmx80m",
 					"-classpath", getCurrentClasspath(),
-					AbstractProcessFailureRecoveryTest.TaskManagerProcessEntryPoint.class.getName(),
+					AbstractTaskManagerProcessFailureRecoveryTest.TaskManagerProcessEntryPoint.class.getName(),
 					String.valueOf(jobManagerPort)
 			};
 
 			// start the first two TaskManager processes
 			taskManagerProcess = new ProcessBuilder(command).start();
-			new AbstractProcessFailureRecoveryTest.PipeForwarder(taskManagerProcess.getErrorStream(), processOutput);
+			new CommonTestUtils.PipeForwarder(taskManagerProcess.getErrorStream(), processOutput);
 			
 			// we wait for the JobManager to have the two TaskManagers available
 			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
deleted file mode 100644
index 054b321..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.recovery;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.UUID;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
-
-import org.junit.Assert;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test for streaming program behaviour in case of TaskManager failure
- * based on {@link AbstractProcessFailureRecoveryTest}.
- *
- * The logic in this test is as follows:
- *  - The source slowly emits records (every 10 msecs) until the test driver
- *    gives the "go" for regular execution
- *  - The "go" is given after the first taskmanager has been killed, so it can only
- *    happen in the recovery run
- *  - The mapper must not be slow, because otherwise the checkpoint barrier cannot pass
- *    the mapper and no checkpoint will be completed before the killing of the first
- *    TaskManager.
- */
-@SuppressWarnings("serial")
-public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailureRecoveryTest {
-
-	private static final int DATA_COUNT = 10000;
-
-	@Override
-	public void testProgram(int jobManagerPort, final File coordinateDir) throws Exception {
-		
-		final File tempCheckpointDir = new File(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH),
-				UUID.randomUUID().toString());
-
-		assertTrue("Cannot create directory for checkpoints", tempCheckpointDir.mkdirs());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment
-									.createRemoteEnvironment("localhost", jobManagerPort);
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-		env.setNumberOfExecutionRetries(1);
-		env.enableCheckpointing(200);
-		
-		env.setStateBackend(new FsStateBackend(tempCheckpointDir.getAbsoluteFile().toURI()));
-
-		DataStream<Long> result = env.addSource(new SleepyDurableGenerateSequence(coordinateDir, DATA_COUNT))
-				// add a non-chained no-op map to test the chain state restore logic
-				.map(new MapFunction<Long, Long>() {
-					@Override
-					public Long map(Long value) throws Exception {
-						return value;
-					}
-				}).startNewChain()
-				// populate the coordinate directory so we can proceed to TaskManager failure
-				.map(new Mapper(coordinateDir));
-
-		//write result to temporary file
-		result.addSink(new CheckpointedSink(DATA_COUNT));
-
-		try {
-			// blocking call until execution is done
-			env.execute();
-
-			// TODO: Figure out why this fails when ran with other tests
-			// Check whether checkpoints have been cleaned up properly
-			// assertDirectoryEmpty(tempCheckpointDir);
-		}
-		finally {
-			// clean up
-			if (tempCheckpointDir.exists()) {
-				FileUtils.deleteDirectory(tempCheckpointDir);
-			}
-		}
-	}
-
-	public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long> 
-			implements Checkpointed<Long> {
-
-		private static final long SLEEP_TIME = 50;
-
-		private final File coordinateDir;
-		private final long end;
-
-		private volatile boolean isRunning = true;
-		
-		private long collected;
-
-		public SleepyDurableGenerateSequence(File coordinateDir, long end) {
-			this.coordinateDir = coordinateDir;
-			this.end = end;
-		}
-
-		@Override
-		public void run(SourceContext<Long> sourceCtx) throws Exception {
-			final Object checkpointLock = sourceCtx.getCheckpointLock();
-
-			RuntimeContext runtimeCtx = getRuntimeContext();
-
-			final long stepSize = runtimeCtx.getNumberOfParallelSubtasks();
-			final long congruence = runtimeCtx.getIndexOfThisSubtask();
-			final long toCollect = (end % stepSize > congruence) ? (end / stepSize + 1) : (end / stepSize);
-
-			final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
-			boolean checkForProceedFile = true;
-
-			while (isRunning && collected < toCollect) {
-				// check if the proceed file exists (then we go full speed)
-				// if not, we always recheck and sleep
-				if (checkForProceedFile) {
-					if (proceedFile.exists()) {
-						checkForProceedFile = false;
-					} else {
-						// otherwise wait so that we make slow progress
-						Thread.sleep(SLEEP_TIME);
-					}
-				}
-
-				synchronized (checkpointLock) {
-					sourceCtx.collect(collected * stepSize + congruence);
-					collected++;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return collected;
-		}
-
-		@Override
-		public void restoreState(Long state) {
-			collected = state;
-		}
-	}
-	
-	public static class Mapper extends RichMapFunction<Long, Long> {
-		private boolean markerCreated = false;
-		private File coordinateDir;
-
-		public Mapper(File coordinateDir) {
-			this.coordinateDir = coordinateDir;
-		}
-
-		@Override
-		public Long map(Long value) throws Exception {
-			if (!markerCreated) {
-				int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
-				touchFile(new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex));
-				markerCreated = true;
-			}
-			return value;
-		}
-	}
-
-	private static class CheckpointedSink extends RichSinkFunction<Long> implements Checkpointed<Long> {
-
-		private long stepSize;
-		private long congruence;
-		private long toCollect;
-		private Long collected = 0L;
-		private long end;
-
-		public CheckpointedSink(long end) {
-			this.end = end;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws IOException {
-			stepSize = getRuntimeContext().getNumberOfParallelSubtasks();
-			congruence = getRuntimeContext().getIndexOfThisSubtask();
-			toCollect = (end % stepSize > congruence) ? (end / stepSize + 1) : (end / stepSize);
-		}
-
-		@Override
-		public void invoke(Long value) throws Exception {
-			long expected = collected * stepSize + congruence;
-
-			Assert.assertTrue("Value did not match expected value. " + expected + " != " + value, value.equals(expected));
-
-			collected++;
-
-			if (collected > toCollect) {
-				Assert.fail("Collected <= toCollect: " + collected + " > " + toCollect);
-			}
-
-		}
-
-		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return collected;
-		}
-
-		@Override
-		public void restoreState(Long state) {
-			collected = state;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
new file mode 100644
index 0000000..173c8ea
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
@@ -0,0 +1,115 @@
+/*
+ * 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.test.recovery;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test the recovery of a simple batch program in the case of TaskManager process failure.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class TaskManagerProcessFailureBatchRecoveryITCase extends AbstractTaskManagerProcessFailureRecoveryTest {
+
+	// --------------------------------------------------------------------------------------------
+	//  Parametrization (run pipelined and batch)
+	// --------------------------------------------------------------------------------------------
+
+	private final ExecutionMode executionMode;
+
+	public TaskManagerProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) {
+		this.executionMode = executionMode;
+	}
+
+	@Parameterized.Parameters
+	public static Collection<Object[]> executionMode() {
+		return Arrays.asList(new Object[][]{
+				{ExecutionMode.PIPELINED},
+				{ExecutionMode.BATCH}});
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Test the program
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort);
+		env.setParallelism(PARALLELISM);
+		env.setNumberOfExecutionRetries(1);
+		env.getConfig().setExecutionMode(executionMode);
+		env.getConfig().disableSysoutLogging();
+
+		final long NUM_ELEMENTS = 100000L;
+		final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)
+
+				// make sure every mapper is involved (no one is skipped because of lazy split assignment)
+				.rebalance()
+						// the majority of the behavior is in the MapFunction
+				.map(new RichMapFunction<Long, Long>() {
+
+					private final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
+
+					private boolean markerCreated = false;
+					private boolean checkForProceedFile = true;
+
+					@Override
+					public Long map(Long value) throws Exception {
+						if (!markerCreated) {
+							int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+							touchFile(new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex));
+							markerCreated = true;
+						}
+
+						// check if the proceed file exists
+						if (checkForProceedFile) {
+							if (proceedFile.exists()) {
+								checkForProceedFile = false;
+							} else {
+								// otherwise wait so that we make slow progress
+								Thread.sleep(100);
+							}
+						}
+						return value;
+					}
+				})
+				.reduce(new ReduceFunction<Long>() {
+					@Override
+					public Long reduce(Long value1, Long value2) {
+						return value1 + value2;
+					}
+				});
+
+		long sum = result.collect().get(0);
+		assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, sum);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
new file mode 100644
index 0000000..aa634f0
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
@@ -0,0 +1,234 @@
+/*
+ * 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.test.recovery;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+
+import org.junit.Assert;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for streaming program behaviour in case of TaskManager failure
+ * based on {@link AbstractTaskManagerProcessFailureRecoveryTest}.
+ *
+ * The logic in this test is as follows:
+ *  - The source slowly emits records (every 10 msecs) until the test driver
+ *    gives the "go" for regular execution
+ *  - The "go" is given after the first taskmanager has been killed, so it can only
+ *    happen in the recovery run
+ *  - The mapper must not be slow, because otherwise the checkpoint barrier cannot pass
+ *    the mapper and no checkpoint will be completed before the killing of the first
+ *    TaskManager.
+ */
+@SuppressWarnings("serial")
+public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTaskManagerProcessFailureRecoveryTest {
+
+	private static final int DATA_COUNT = 10000;
+
+	@Override
+	public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) throws Exception {
+
+		final File tempCheckpointDir = new File(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH),
+				UUID.randomUUID().toString());
+
+		assertTrue("Cannot create directory for checkpoints", tempCheckpointDir.mkdirs());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment
+				.createRemoteEnvironment("localhost", jobManagerPort);
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+		env.setNumberOfExecutionRetries(1);
+		env.enableCheckpointing(200);
+		
+		env.setStateBackend(new FsStateBackend(tempCheckpointDir.getAbsoluteFile().toURI()));
+
+		DataStream<Long> result = env.addSource(new SleepyDurableGenerateSequence(coordinateDir, DATA_COUNT))
+				// add a non-chained no-op map to test the chain state restore logic
+				.map(new MapFunction<Long, Long>() {
+					@Override
+					public Long map(Long value) throws Exception {
+						return value;
+					}
+				}).startNewChain()
+						// populate the coordinate directory so we can proceed to TaskManager failure
+				.map(new Mapper(coordinateDir));
+
+		//write result to temporary file
+		result.addSink(new CheckpointedSink(DATA_COUNT));
+
+		try {
+			// blocking call until execution is done
+			env.execute();
+
+			// TODO: Figure out why this fails when ran with other tests
+			// Check whether checkpoints have been cleaned up properly
+			// assertDirectoryEmpty(tempCheckpointDir);
+		}
+		finally {
+			// clean up
+			if (tempCheckpointDir.exists()) {
+				FileUtils.deleteDirectory(tempCheckpointDir);
+			}
+		}
+	}
+
+	public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long> 
+			implements Checkpointed<Long> {
+
+		private static final long SLEEP_TIME = 50;
+
+		private final File coordinateDir;
+		private final long end;
+
+		private volatile boolean isRunning = true;
+
+		private long collected;
+
+		public SleepyDurableGenerateSequence(File coordinateDir, long end) {
+			this.coordinateDir = coordinateDir;
+			this.end = end;
+		}
+
+		@Override
+		public void run(SourceContext<Long> sourceCtx) throws Exception {
+			final Object checkpointLock = sourceCtx.getCheckpointLock();
+
+			RuntimeContext runtimeCtx = getRuntimeContext();
+
+			final long stepSize = runtimeCtx.getNumberOfParallelSubtasks();
+			final long congruence = runtimeCtx.getIndexOfThisSubtask();
+			final long toCollect = (end % stepSize > congruence) ? (end / stepSize + 1) : (end / stepSize);
+
+			final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
+			boolean checkForProceedFile = true;
+
+			while (isRunning && collected < toCollect) {
+				// check if the proceed file exists (then we go full speed)
+				// if not, we always recheck and sleep
+				if (checkForProceedFile) {
+					if (proceedFile.exists()) {
+						checkForProceedFile = false;
+					} else {
+						// otherwise wait so that we make slow progress
+						Thread.sleep(SLEEP_TIME);
+					}
+				}
+
+				synchronized (checkpointLock) {
+					sourceCtx.collect(collected * stepSize + congruence);
+					collected++;
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+			return collected;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			collected = state;
+		}
+	}
+
+	public static class Mapper extends RichMapFunction<Long, Long> {
+		private boolean markerCreated = false;
+		private File coordinateDir;
+
+		public Mapper(File coordinateDir) {
+			this.coordinateDir = coordinateDir;
+		}
+
+		@Override
+		public Long map(Long value) throws Exception {
+			if (!markerCreated) {
+				int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+				touchFile(new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex));
+				markerCreated = true;
+			}
+			return value;
+		}
+	}
+
+	private static class CheckpointedSink extends RichSinkFunction<Long> implements Checkpointed<Long> {
+
+		private long stepSize;
+		private long congruence;
+		private long toCollect;
+		private Long collected = 0L;
+		private long end;
+
+		public CheckpointedSink(long end) {
+			this.end = end;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws IOException {
+			stepSize = getRuntimeContext().getNumberOfParallelSubtasks();
+			congruence = getRuntimeContext().getIndexOfThisSubtask();
+			toCollect = (end % stepSize > congruence) ? (end / stepSize + 1) : (end / stepSize);
+		}
+
+		@Override
+		public void invoke(Long value) throws Exception {
+			long expected = collected * stepSize + congruence;
+
+			Assert.assertTrue("Value did not match expected value. " + expected + " != " + value, value.equals(expected));
+
+			collected++;
+
+			if (collected > toCollect) {
+				Assert.fail("Collected <= toCollect: " + collected + " > " + toCollect);
+			}
+
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return collected;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			collected = state;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
index 6035c45..ed2113a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.runtime.leaderelection;
 import akka.actor.ActorSystem;
 import akka.actor.Kill;
 import akka.actor.PoisonPill;
+import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.client.JobClient;
@@ -39,17 +40,41 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
 import org.junit.Test;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-import static org.junit.Assert.*;
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 	private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION();
 
+	private static final File tempDirectory;
+
+	static {
+		try {
+			tempDirectory = org.apache.flink.runtime.testutils
+					.CommonTestUtils.createTempDirectory();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Test setup failed", e);
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (tempDirectory != null) {
+			FileUtils.deleteDirectory(tempDirectory);
+		}
+	}
+
 	/**
 	 * Tests that the TaskManagers successfully register at the new leader once the old leader
 	 * is terminated.
@@ -64,13 +89,15 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
+		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
+		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath());
 
 		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
 
 		try {
 			cluster.start();
 
-			for(int i = 0; i < numJMs; i++) {
+			for (int i = 0; i < numJMs; i++) {
 				ActorGateway leadingJM = cluster.getLeaderGateway(timeout);
 
 				cluster.waitForTaskManagersToBeRegisteredAtJobManager(leadingJM.actor());
@@ -86,7 +113,8 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 				cluster.clearLeader();
 				leadingJM.tell(PoisonPill.getInstance());
 			}
-		} finally {
+		}
+		finally {
 			cluster.stop();
 		}
 	}
@@ -110,6 +138,13 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
+		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
+		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath());
+
+		// @TODO @tillrohrmann temporary "disable" recovery, because currently the client does
+		// not need to resubmit a failed job to a new leader. Should we keep this test and
+		// disable recovery fully or will this be subsumed by the real client changes anyways?
+		configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, timeout.toString());
 
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
 
@@ -152,7 +187,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 			thread.start();
 
 			// Kill all JobManager except for two
-			for(int i = 0; i < numJMs - 2; i++) {
+			for (int i = 0; i < numJMs - 2; i++) {
 				ActorGateway jm = cluster.getLeaderGateway(timeout);
 
 				cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor());
@@ -184,17 +219,18 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 			thread.join(timeout.toMillis());
 
-			if(thread.isAlive()) {
+			if (thread.isAlive()) {
 				jobSubmission.finished = true;
 				fail("The job submission thread did not stop (meaning it did not succeeded in" +
 						"executing the test job.");
 			}
-		} finally {
+		}
+		finally {
 			if (clientActorSystem != null) {
 				cluster.shutdownJobClientActorSystem(clientActorSystem);
 			}
 
-			if(thread != null && thread.isAlive() && jobSubmission != null) {
+			if (thread != null && thread.isAlive() && jobSubmission != null) {
 				jobSubmission.finished = true;
 			}
 			cluster.stop();
@@ -219,7 +255,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 		@Override
 		public void run() {
-			while(!finished) {
+			while (!finished) {
 				try {
 					LeaderRetrievalService lrService =
 							LeaderRetrievalUtils.createLeaderRetrievalService(
@@ -240,11 +276,14 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 							getClass().getClassLoader());
 
 					finished = true;
-				} catch (JobExecutionException e) {
+				}
+				catch (JobExecutionException e) {
 					// This was expected, so just try again to submit the job
-				} catch (LeaderRetrievalException e) {
+				}
+				catch (LeaderRetrievalException e) {
 					// This can also happen, so just try again to submit the job
-				} catch (Exception e) {
+				}
+				catch (Exception e) {
 					// This was not expected... fail the test case
 					e.printStackTrace();
 					fail("Caught unexpected exception in job submission test case.");

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
index 61eb6a5..4ada21e 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
@@ -29,8 +29,9 @@ import grizzled.slf4j.Logger
 import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants}
 import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
 import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager}
 import org.apache.flink.runtime.leaderelection.LeaderElectionService
 import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatus, CurrentJobStatus,
 JobNotFound}
@@ -88,7 +89,9 @@ class YarnJobManager(
     delayBetweenRetries: Long,
     timeout: FiniteDuration,
     mode: StreamingMode,
-    leaderElectionService: LeaderElectionService)
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory)
   extends JobManager(
     flinkConfiguration,
     executionContext,
@@ -100,7 +103,9 @@ class YarnJobManager(
     delayBetweenRetries,
     timeout,
     mode,
-    leaderElectionService) {
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory) {
 
   import context._
   import scala.collection.JavaConverters._


[18/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/js/vendor.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/vendor.js b/flink-runtime-web/src/main/resources/web/js/vendor.js
new file mode 100644
index 0000000..52bc81c
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/js/vendor.js
@@ -0,0 +1,81087 @@
+/*!
+ * jQuery JavaScript Library v2.1.4
+ * http://jquery.com/
+ *
+ * Includes Sizzle.js
+ * http://sizzlejs.com/
+ *
+ * Copyright 2005, 2014 jQuery Foundation, Inc. and other contributors
+ * Released under the MIT license
+ * http://jquery.org/license
+ *
+ * Date: 2015-04-28T16:01Z
+ */
+
+(function( global, factory ) {
+
+	if ( typeof module === "object" && typeof module.exports === "object" ) {
+		// For CommonJS and CommonJS-like environments where a proper `window`
+		// is present, execute the factory and get jQuery.
+		// For environments that do not have a `window` with a `document`
+		// (such as Node.js), expose a factory as module.exports.
+		// This accentuates the need for the creation of a real `window`.
+		// e.g. var jQuery = require("jquery")(window);
+		// See ticket #14549 for more info.
+		module.exports = global.document ?
+			factory( global, true ) :
+			function( w ) {
+				if ( !w.document ) {
+					throw new Error( "jQuery requires a window with a document" );
+				}
+				return factory( w );
+			};
+	} else {
+		factory( global );
+	}
+
+// Pass this if window is not defined yet
+}(typeof window !== "undefined" ? window : this, function( window, noGlobal ) {
+
+// Support: Firefox 18+
+// Can't be in strict mode, several libs including ASP.NET trace
+// the stack via arguments.caller.callee and Firefox dies if
+// you try to trace through "use strict" call chains. (#13335)
+//
+
+var arr = [];
+
+var slice = arr.slice;
+
+var concat = arr.concat;
+
+var push = arr.push;
+
+var indexOf = arr.indexOf;
+
+var class2type = {};
+
+var toString = class2type.toString;
+
+var hasOwn = class2type.hasOwnProperty;
+
+var support = {};
+
+
+
+var
+	// Use the correct document accordingly with window argument (sandbox)
+	document = window.document,
+
+	version = "2.1.4",
+
+	// Define a local copy of jQuery
+	jQuery = function( selector, context ) {
+		// The jQuery object is actually just the init constructor 'enhanced'
+		// Need init if jQuery is called (just allow error to be thrown if not included)
+		return new jQuery.fn.init( selector, context );
+	},
+
+	// Support: Android<4.1
+	// Make sure we trim BOM and NBSP
+	rtrim = /^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,
+
+	// Matches dashed string for camelizing
+	rmsPrefix = /^-ms-/,
+	rdashAlpha = /-([\da-z])/gi,
+
+	// Used by jQuery.camelCase as callback to replace()
+	fcamelCase = function( all, letter ) {
+		return letter.toUpperCase();
+	};
+
+jQuery.fn = jQuery.prototype = {
+	// The current version of jQuery being used
+	jquery: version,
+
+	constructor: jQuery,
+
+	// Start with an empty selector
+	selector: "",
+
+	// The default length of a jQuery object is 0
+	length: 0,
+
+	toArray: function() {
+		return slice.call( this );
+	},
+
+	// Get the Nth element in the matched element set OR
+	// Get the whole matched element set as a clean array
+	get: function( num ) {
+		return num != null ?
+
+			// Return just the one element from the set
+			( num < 0 ? this[ num + this.length ] : this[ num ] ) :
+
+			// Return all the elements in a clean array
+			slice.call( this );
+	},
+
+	// Take an array of elements and push it onto the stack
+	// (returning the new matched element set)
+	pushStack: function( elems ) {
+
+		// Build a new jQuery matched element set
+		var ret = jQuery.merge( this.constructor(), elems );
+
+		// Add the old object onto the stack (as a reference)
+		ret.prevObject = this;
+		ret.context = this.context;
+
+		// Return the newly-formed element set
+		return ret;
+	},
+
+	// Execute a callback for every element in the matched set.
+	// (You can seed the arguments with an array of args, but this is
+	// only used internally.)
+	each: function( callback, args ) {
+		return jQuery.each( this, callback, args );
+	},
+
+	map: function( callback ) {
+		return this.pushStack( jQuery.map(this, function( elem, i ) {
+			return callback.call( elem, i, elem );
+		}));
+	},
+
+	slice: function() {
+		return this.pushStack( slice.apply( this, arguments ) );
+	},
+
+	first: function() {
+		return this.eq( 0 );
+	},
+
+	last: function() {
+		return this.eq( -1 );
+	},
+
+	eq: function( i ) {
+		var len = this.length,
+			j = +i + ( i < 0 ? len : 0 );
+		return this.pushStack( j >= 0 && j < len ? [ this[j] ] : [] );
+	},
+
+	end: function() {
+		return this.prevObject || this.constructor(null);
+	},
+
+	// For internal use only.
+	// Behaves like an Array's method, not like a jQuery method.
+	push: push,
+	sort: arr.sort,
+	splice: arr.splice
+};
+
+jQuery.extend = jQuery.fn.extend = function() {
+	var options, name, src, copy, copyIsArray, clone,
+		target = arguments[0] || {},
+		i = 1,
+		length = arguments.length,
+		deep = false;
+
+	// Handle a deep copy situation
+	if ( typeof target === "boolean" ) {
+		deep = target;
+
+		// Skip the boolean and the target
+		target = arguments[ i ] || {};
+		i++;
+	}
+
+	// Handle case when target is a string or something (possible in deep copy)
+	if ( typeof target !== "object" && !jQuery.isFunction(target) ) {
+		target = {};
+	}
+
+	// Extend jQuery itself if only one argument is passed
+	if ( i === length ) {
+		target = this;
+		i--;
+	}
+
+	for ( ; i < length; i++ ) {
+		// Only deal with non-null/undefined values
+		if ( (options = arguments[ i ]) != null ) {
+			// Extend the base object
+			for ( name in options ) {
+				src = target[ name ];
+				copy = options[ name ];
+
+				// Prevent never-ending loop
+				if ( target === copy ) {
+					continue;
+				}
+
+				// Recurse if we're merging plain objects or arrays
+				if ( deep && copy && ( jQuery.isPlainObject(copy) || (copyIsArray = jQuery.isArray(copy)) ) ) {
+					if ( copyIsArray ) {
+						copyIsArray = false;
+						clone = src && jQuery.isArray(src) ? src : [];
+
+					} else {
+						clone = src && jQuery.isPlainObject(src) ? src : {};
+					}
+
+					// Never move original objects, clone them
+					target[ name ] = jQuery.extend( deep, clone, copy );
+
+				// Don't bring in undefined values
+				} else if ( copy !== undefined ) {
+					target[ name ] = copy;
+				}
+			}
+		}
+	}
+
+	// Return the modified object
+	return target;
+};
+
+jQuery.extend({
+	// Unique for each copy of jQuery on the page
+	expando: "jQuery" + ( version + Math.random() ).replace( /\D/g, "" ),
+
+	// Assume jQuery is ready without the ready module
+	isReady: true,
+
+	error: function( msg ) {
+		throw new Error( msg );
+	},
+
+	noop: function() {},
+
+	isFunction: function( obj ) {
+		return jQuery.type(obj) === "function";
+	},
+
+	isArray: Array.isArray,
+
+	isWindow: function( obj ) {
+		return obj != null && obj === obj.window;
+	},
+
+	isNumeric: function( obj ) {
+		// parseFloat NaNs numeric-cast false positives (null|true|false|"")
+		// ...but misinterprets leading-number strings, particularly hex literals ("0x...")
+		// subtraction forces infinities to NaN
+		// adding 1 corrects loss of precision from parseFloat (#15100)
+		return !jQuery.isArray( obj ) && (obj - parseFloat( obj ) + 1) >= 0;
+	},
+
+	isPlainObject: function( obj ) {
+		// Not plain objects:
+		// - Any object or value whose internal [[Class]] property is not "[object Object]"
+		// - DOM nodes
+		// - window
+		if ( jQuery.type( obj ) !== "object" || obj.nodeType || jQuery.isWindow( obj ) ) {
+			return false;
+		}
+
+		if ( obj.constructor &&
+				!hasOwn.call( obj.constructor.prototype, "isPrototypeOf" ) ) {
+			return false;
+		}
+
+		// If the function hasn't returned already, we're confident that
+		// |obj| is a plain object, created by {} or constructed with new Object
+		return true;
+	},
+
+	isEmptyObject: function( obj ) {
+		var name;
+		for ( name in obj ) {
+			return false;
+		}
+		return true;
+	},
+
+	type: function( obj ) {
+		if ( obj == null ) {
+			return obj + "";
+		}
+		// Support: Android<4.0, iOS<6 (functionish RegExp)
+		return typeof obj === "object" || typeof obj === "function" ?
+			class2type[ toString.call(obj) ] || "object" :
+			typeof obj;
+	},
+
+	// Evaluates a script in a global context
+	globalEval: function( code ) {
+		var script,
+			indirect = eval;
+
+		code = jQuery.trim( code );
+
+		if ( code ) {
+			// If the code includes a valid, prologue position
+			// strict mode pragma, execute code by injecting a
+			// script tag into the document.
+			if ( code.indexOf("use strict") === 1 ) {
+				script = document.createElement("script");
+				script.text = code;
+				document.head.appendChild( script ).parentNode.removeChild( script );
+			} else {
+			// Otherwise, avoid the DOM node creation, insertion
+			// and removal by using an indirect global eval
+				indirect( code );
+			}
+		}
+	},
+
+	// Convert dashed to camelCase; used by the css and data modules
+	// Support: IE9-11+
+	// Microsoft forgot to hump their vendor prefix (#9572)
+	camelCase: function( string ) {
+		return string.replace( rmsPrefix, "ms-" ).replace( rdashAlpha, fcamelCase );
+	},
+
+	nodeName: function( elem, name ) {
+		return elem.nodeName && elem.nodeName.toLowerCase() === name.toLowerCase();
+	},
+
+	// args is for internal usage only
+	each: function( obj, callback, args ) {
+		var value,
+			i = 0,
+			length = obj.length,
+			isArray = isArraylike( obj );
+
+		if ( args ) {
+			if ( isArray ) {
+				for ( ; i < length; i++ ) {
+					value = callback.apply( obj[ i ], args );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			} else {
+				for ( i in obj ) {
+					value = callback.apply( obj[ i ], args );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			}
+
+		// A special, fast, case for the most common use of each
+		} else {
+			if ( isArray ) {
+				for ( ; i < length; i++ ) {
+					value = callback.call( obj[ i ], i, obj[ i ] );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			} else {
+				for ( i in obj ) {
+					value = callback.call( obj[ i ], i, obj[ i ] );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			}
+		}
+
+		return obj;
+	},
+
+	// Support: Android<4.1
+	trim: function( text ) {
+		return text == null ?
+			"" :
+			( text + "" ).replace( rtrim, "" );
+	},
+
+	// results is for internal usage only
+	makeArray: function( arr, results ) {
+		var ret = results || [];
+
+		if ( arr != null ) {
+			if ( isArraylike( Object(arr) ) ) {
+				jQuery.merge( ret,
+					typeof arr === "string" ?
+					[ arr ] : arr
+				);
+			} else {
+				push.call( ret, arr );
+			}
+		}
+
+		return ret;
+	},
+
+	inArray: function( elem, arr, i ) {
+		return arr == null ? -1 : indexOf.call( arr, elem, i );
+	},
+
+	merge: function( first, second ) {
+		var len = +second.length,
+			j = 0,
+			i = first.length;
+
+		for ( ; j < len; j++ ) {
+			first[ i++ ] = second[ j ];
+		}
+
+		first.length = i;
+
+		return first;
+	},
+
+	grep: function( elems, callback, invert ) {
+		var callbackInverse,
+			matches = [],
+			i = 0,
+			length = elems.length,
+			callbackExpect = !invert;
+
+		// Go through the array, only saving the items
+		// that pass the validator function
+		for ( ; i < length; i++ ) {
+			callbackInverse = !callback( elems[ i ], i );
+			if ( callbackInverse !== callbackExpect ) {
+				matches.push( elems[ i ] );
+			}
+		}
+
+		return matches;
+	},
+
+	// arg is for internal usage only
+	map: function( elems, callback, arg ) {
+		var value,
+			i = 0,
+			length = elems.length,
+			isArray = isArraylike( elems ),
+			ret = [];
+
+		// Go through the array, translating each of the items to their new values
+		if ( isArray ) {
+			for ( ; i < length; i++ ) {
+				value = callback( elems[ i ], i, arg );
+
+				if ( value != null ) {
+					ret.push( value );
+				}
+			}
+
+		// Go through every key on the object,
+		} else {
+			for ( i in elems ) {
+				value = callback( elems[ i ], i, arg );
+
+				if ( value != null ) {
+					ret.push( value );
+				}
+			}
+		}
+
+		// Flatten any nested arrays
+		return concat.apply( [], ret );
+	},
+
+	// A global GUID counter for objects
+	guid: 1,
+
+	// Bind a function to a context, optionally partially applying any
+	// arguments.
+	proxy: function( fn, context ) {
+		var tmp, args, proxy;
+
+		if ( typeof context === "string" ) {
+			tmp = fn[ context ];
+			context = fn;
+			fn = tmp;
+		}
+
+		// Quick check to determine if target is callable, in the spec
+		// this throws a TypeError, but we will just return undefined.
+		if ( !jQuery.isFunction( fn ) ) {
+			return undefined;
+		}
+
+		// Simulated bind
+		args = slice.call( arguments, 2 );
+		proxy = function() {
+			return fn.apply( context || this, args.concat( slice.call( arguments ) ) );
+		};
+
+		// Set the guid of unique handler to the same of original handler, so it can be removed
+		proxy.guid = fn.guid = fn.guid || jQuery.guid++;
+
+		return proxy;
+	},
+
+	now: Date.now,
+
+	// jQuery.support is not used in Core but other projects attach their
+	// properties to it so it needs to exist.
+	support: support
+});
+
+// Populate the class2type map
+jQuery.each("Boolean Number String Function Array Date RegExp Object Error".split(" "), function(i, name) {
+	class2type[ "[object " + name + "]" ] = name.toLowerCase();
+});
+
+function isArraylike( obj ) {
+
+	// Support: iOS 8.2 (not reproducible in simulator)
+	// `in` check used to prevent JIT error (gh-2145)
+	// hasOwn isn't used here due to false negatives
+	// regarding Nodelist length in IE
+	var length = "length" in obj && obj.length,
+		type = jQuery.type( obj );
+
+	if ( type === "function" || jQuery.isWindow( obj ) ) {
+		return false;
+	}
+
+	if ( obj.nodeType === 1 && length ) {
+		return true;
+	}
+
+	return type === "array" || length === 0 ||
+		typeof length === "number" && length > 0 && ( length - 1 ) in obj;
+}
+var Sizzle =
+/*!
+ * Sizzle CSS Selector Engine v2.2.0-pre
+ * http://sizzlejs.com/
+ *
+ * Copyright 2008, 2014 jQuery Foundation, Inc. and other contributors
+ * Released under the MIT license
+ * http://jquery.org/license
+ *
+ * Date: 2014-12-16
+ */
+(function( window ) {
+
+var i,
+	support,
+	Expr,
+	getText,
+	isXML,
+	tokenize,
+	compile,
+	select,
+	outermostContext,
+	sortInput,
+	hasDuplicate,
+
+	// Local document vars
+	setDocument,
+	document,
+	docElem,
+	documentIsHTML,
+	rbuggyQSA,
+	rbuggyMatches,
+	matches,
+	contains,
+
+	// Instance-specific data
+	expando = "sizzle" + 1 * new Date(),
+	preferredDoc = window.document,
+	dirruns = 0,
+	done = 0,
+	classCache = createCache(),
+	tokenCache = createCache(),
+	compilerCache = createCache(),
+	sortOrder = function( a, b ) {
+		if ( a === b ) {
+			hasDuplicate = true;
+		}
+		return 0;
+	},
+
+	// General-purpose constants
+	MAX_NEGATIVE = 1 << 31,
+
+	// Instance methods
+	hasOwn = ({}).hasOwnProperty,
+	arr = [],
+	pop = arr.pop,
+	push_native = arr.push,
+	push = arr.push,
+	slice = arr.slice,
+	// Use a stripped-down indexOf as it's faster than native
+	// http://jsperf.com/thor-indexof-vs-for/5
+	indexOf = function( list, elem ) {
+		var i = 0,
+			len = list.length;
+		for ( ; i < len; i++ ) {
+			if ( list[i] === elem ) {
+				return i;
+			}
+		}
+		return -1;
+	},
+
+	booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",
+
+	// Regular expressions
+
+	// Whitespace characters http://www.w3.org/TR/css3-selectors/#whitespace
+	whitespace = "[\\x20\\t\\r\\n\\f]",
+	// http://www.w3.org/TR/css3-syntax/#characters
+	characterEncoding = "(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",
+
+	// Loosely modeled on CSS identifier characters
+	// An unquoted value should be a CSS identifier http://www.w3.org/TR/css3-selectors/#attribute-selectors
+	// Proper syntax: http://www.w3.org/TR/CSS21/syndata.html#value-def-identifier
+	identifier = characterEncoding.replace( "w", "w#" ),
+
+	// Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
+	attributes = "\\[" + whitespace + "*(" + characterEncoding + ")(?:" + whitespace +
+		// Operator (capture 2)
+		"*([*^$|!~]?=)" + whitespace +
+		// "Attribute values must be CSS identifiers [capture 5] or strings [capture 3 or capture 4]"
+		"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + whitespace +
+		"*\\]",
+
+	pseudos = ":(" + characterEncoding + ")(?:\\((" +
+		// To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
+		// 1. quoted (capture 3; capture 4 or capture 5)
+		"('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
+		// 2. simple (capture 6)
+		"((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
+		// 3. anything else (capture 2)
+		".*" +
+		")\\)|)",
+
+	// Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
+	rwhitespace = new RegExp( whitespace + "+", "g" ),
+	rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + whitespace + "+$", "g" ),
+
+	rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
+	rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + "*" ),
+
+	rattributeQuotes = new RegExp( "=" + whitespace + "*([^\\]'\"]*?)" + whitespace + "*\\]", "g" ),
+
+	rpseudo = new RegExp( pseudos ),
+	ridentifier = new RegExp( "^" + identifier + "$" ),
+
+	matchExpr = {
+		"ID": new RegExp( "^#(" + characterEncoding + ")" ),
+		"CLASS": new RegExp( "^\\.(" + characterEncoding + ")" ),
+		"TAG": new RegExp( "^(" + characterEncoding.replace( "w", "w*" ) + ")" ),
+		"ATTR": new RegExp( "^" + attributes ),
+		"PSEUDO": new RegExp( "^" + pseudos ),
+		"CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + whitespace +
+			"*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + whitespace +
+			"*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
+		"bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
+		// For use in libraries implementing .is()
+		// We use this for POS matching in `select`
+		"needsContext": new RegExp( "^" + whitespace + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" +
+			whitespace + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
+	},
+
+	rinputs = /^(?:input|select|textarea|button)$/i,
+	rheader = /^h\d$/i,
+
+	rnative = /^[^{]+\{\s*\[native \w/,
+
+	// Easily-parseable/retrievable ID or TAG or CLASS selectors
+	rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
+
+	rsibling = /[+~]/,
+	rescape = /'|\\/g,
+
+	// CSS escapes http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
+	runescape = new RegExp( "\\\\([\\da-f]{1,6}" + whitespace + "?|(" + whitespace + ")|.)", "ig" ),
+	funescape = function( _, escaped, escapedWhitespace ) {
+		var high = "0x" + escaped - 0x10000;
+		// NaN means non-codepoint
+		// Support: Firefox<24
+		// Workaround erroneous numeric interpretation of +"0x"
+		return high !== high || escapedWhitespace ?
+			escaped :
+			high < 0 ?
+				// BMP codepoint
+				String.fromCharCode( high + 0x10000 ) :
+				// Supplemental Plane codepoint (surrogate pair)
+				String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
+	},
+
+	// Used for iframes
+	// See setDocument()
+	// Removing the function wrapper causes a "Permission Denied"
+	// error in IE
+	unloadHandler = function() {
+		setDocument();
+	};
+
+// Optimize for push.apply( _, NodeList )
+try {
+	push.apply(
+		(arr = slice.call( preferredDoc.childNodes )),
+		preferredDoc.childNodes
+	);
+	// Support: Android<4.0
+	// Detect silently failing push.apply
+	arr[ preferredDoc.childNodes.length ].nodeType;
+} catch ( e ) {
+	push = { apply: arr.length ?
+
+		// Leverage slice if possible
+		function( target, els ) {
+			push_native.apply( target, slice.call(els) );
+		} :
+
+		// Support: IE<9
+		// Otherwise append directly
+		function( target, els ) {
+			var j = target.length,
+				i = 0;
+			// Can't trust NodeList.length
+			while ( (target[j++] = els[i++]) ) {}
+			target.length = j - 1;
+		}
+	};
+}
+
+function Sizzle( selector, context, results, seed ) {
+	var match, elem, m, nodeType,
+		// QSA vars
+		i, groups, old, nid, newContext, newSelector;
+
+	if ( ( context ? context.ownerDocument || context : preferredDoc ) !== document ) {
+		setDocument( context );
+	}
+
+	context = context || document;
+	results = results || [];
+	nodeType = context.nodeType;
+
+	if ( typeof selector !== "string" || !selector ||
+		nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
+
+		return results;
+	}
+
+	if ( !seed && documentIsHTML ) {
+
+		// Try to shortcut find operations when possible (e.g., not under DocumentFragment)
+		if ( nodeType !== 11 && (match = rquickExpr.exec( selector )) ) {
+			// Speed-up: Sizzle("#ID")
+			if ( (m = match[1]) ) {
+				if ( nodeType === 9 ) {
+					elem = context.getElementById( m );
+					// Check parentNode to catch when Blackberry 4.6 returns
+					// nodes that are no longer in the document (jQuery #6963)
+					if ( elem && elem.parentNode ) {
+						// Handle the case where IE, Opera, and Webkit return items
+						// by name instead of ID
+						if ( elem.id === m ) {
+							results.push( elem );
+							return results;
+						}
+					} else {
+						return results;
+					}
+				} else {
+					// Context is not a document
+					if ( context.ownerDocument && (elem = context.ownerDocument.getElementById( m )) &&
+						contains( context, elem ) && elem.id === m ) {
+						results.push( elem );
+						return results;
+					}
+				}
+
+			// Speed-up: Sizzle("TAG")
+			} else if ( match[2] ) {
+				push.apply( results, context.getElementsByTagName( selector ) );
+				return results;
+
+			// Speed-up: Sizzle(".CLASS")
+			} else if ( (m = match[3]) && support.getElementsByClassName ) {
+				push.apply( results, context.getElementsByClassName( m ) );
+				return results;
+			}
+		}
+
+		// QSA path
+		if ( support.qsa && (!rbuggyQSA || !rbuggyQSA.test( selector )) ) {
+			nid = old = expando;
+			newContext = context;
+			newSelector = nodeType !== 1 && selector;
+
+			// qSA works strangely on Element-rooted queries
+			// We can work around this by specifying an extra ID on the root
+			// and working up from there (Thanks to Andrew Dupont for the technique)
+			// IE 8 doesn't work on object elements
+			if ( nodeType === 1 && context.nodeName.toLowerCase() !== "object" ) {
+				groups = tokenize( selector );
+
+				if ( (old = context.getAttribute("id")) ) {
+					nid = old.replace( rescape, "\\$&" );
+				} else {
+					context.setAttribute( "id", nid );
+				}
+				nid = "[id='" + nid + "'] ";
+
+				i = groups.length;
+				while ( i-- ) {
+					groups[i] = nid + toSelector( groups[i] );
+				}
+				newContext = rsibling.test( selector ) && testContext( context.parentNode ) || context;
+				newSelector = groups.join(",");
+			}
+
+			if ( newSelector ) {
+				try {
+					push.apply( results,
+						newContext.querySelectorAll( newSelector )
+					);
+					return results;
+				} catch(qsaError) {
+				} finally {
+					if ( !old ) {
+						context.removeAttribute("id");
+					}
+				}
+			}
+		}
+	}
+
+	// All others
+	return select( selector.replace( rtrim, "$1" ), context, results, seed );
+}
+
+/**
+ * Create key-value caches of limited size
+ * @returns {Function(string, Object)} Returns the Object data after storing it on itself with
+ *	property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
+ *	deleting the oldest entry
+ */
+function createCache() {
+	var keys = [];
+
+	function cache( key, value ) {
+		// Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
+		if ( keys.push( key + " " ) > Expr.cacheLength ) {
+			// Only keep the most recent entries
+			delete cache[ keys.shift() ];
+		}
+		return (cache[ key + " " ] = value);
+	}
+	return cache;
+}
+
+/**
+ * Mark a function for special use by Sizzle
+ * @param {Function} fn The function to mark
+ */
+function markFunction( fn ) {
+	fn[ expando ] = true;
+	return fn;
+}
+
+/**
+ * Support testing using an element
+ * @param {Function} fn Passed the created div and expects a boolean result
+ */
+function assert( fn ) {
+	var div = document.createElement("div");
+
+	try {
+		return !!fn( div );
+	} catch (e) {
+		return false;
+	} finally {
+		// Remove from its parent by default
+		if ( div.parentNode ) {
+			div.parentNode.removeChild( div );
+		}
+		// release memory in IE
+		div = null;
+	}
+}
+
+/**
+ * Adds the same handler for all of the specified attrs
+ * @param {String} attrs Pipe-separated list of attributes
+ * @param {Function} handler The method that will be applied
+ */
+function addHandle( attrs, handler ) {
+	var arr = attrs.split("|"),
+		i = attrs.length;
+
+	while ( i-- ) {
+		Expr.attrHandle[ arr[i] ] = handler;
+	}
+}
+
+/**
+ * Checks document order of two siblings
+ * @param {Element} a
+ * @param {Element} b
+ * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
+ */
+function siblingCheck( a, b ) {
+	var cur = b && a,
+		diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
+			( ~b.sourceIndex || MAX_NEGATIVE ) -
+			( ~a.sourceIndex || MAX_NEGATIVE );
+
+	// Use IE sourceIndex if available on both nodes
+	if ( diff ) {
+		return diff;
+	}
+
+	// Check if b follows a
+	if ( cur ) {
+		while ( (cur = cur.nextSibling) ) {
+			if ( cur === b ) {
+				return -1;
+			}
+		}
+	}
+
+	return a ? 1 : -1;
+}
+
+/**
+ * Returns a function to use in pseudos for input types
+ * @param {String} type
+ */
+function createInputPseudo( type ) {
+	return function( elem ) {
+		var name = elem.nodeName.toLowerCase();
+		return name === "input" && elem.type === type;
+	};
+}
+
+/**
+ * Returns a function to use in pseudos for buttons
+ * @param {String} type
+ */
+function createButtonPseudo( type ) {
+	return function( elem ) {
+		var name = elem.nodeName.toLowerCase();
+		return (name === "input" || name === "button") && elem.type === type;
+	};
+}
+
+/**
+ * Returns a function to use in pseudos for positionals
+ * @param {Function} fn
+ */
+function createPositionalPseudo( fn ) {
+	return markFunction(function( argument ) {
+		argument = +argument;
+		return markFunction(function( seed, matches ) {
+			var j,
+				matchIndexes = fn( [], seed.length, argument ),
+				i = matchIndexes.length;
+
+			// Match elements found at the specified indexes
+			while ( i-- ) {
+				if ( seed[ (j = matchIndexes[i]) ] ) {
+					seed[j] = !(matches[j] = seed[j]);
+				}
+			}
+		});
+	});
+}
+
+/**
+ * Checks a node for validity as a Sizzle context
+ * @param {Element|Object=} context
+ * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
+ */
+function testContext( context ) {
+	return context && typeof context.getElementsByTagName !== "undefined" && context;
+}
+
+// Expose support vars for convenience
+support = Sizzle.support = {};
+
+/**
+ * Detects XML nodes
+ * @param {Element|Object} elem An element or a document
+ * @returns {Boolean} True iff elem is a non-HTML XML node
+ */
+isXML = Sizzle.isXML = function( elem ) {
+	// documentElement is verified for cases where it doesn't yet exist
+	// (such as loading iframes in IE - #4833)
+	var documentElement = elem && (elem.ownerDocument || elem).documentElement;
+	return documentElement ? documentElement.nodeName !== "HTML" : false;
+};
+
+/**
+ * Sets document-related variables once based on the current document
+ * @param {Element|Object} [doc] An element or document object to use to set the document
+ * @returns {Object} Returns the current document
+ */
+setDocument = Sizzle.setDocument = function( node ) {
+	var hasCompare, parent,
+		doc = node ? node.ownerDocument || node : preferredDoc;
+
+	// If no document and documentElement is available, return
+	if ( doc === document || doc.nodeType !== 9 || !doc.documentElement ) {
+		return document;
+	}
+
+	// Set our document
+	document = doc;
+	docElem = doc.documentElement;
+	parent = doc.defaultView;
+
+	// Support: IE>8
+	// If iframe document is assigned to "document" variable and if iframe has been reloaded,
+	// IE will throw "permission denied" error when accessing "document" variable, see jQuery #13936
+	// IE6-8 do not support the defaultView property so parent will be undefined
+	if ( parent && parent !== parent.top ) {
+		// IE11 does not have attachEvent, so all must suffer
+		if ( parent.addEventListener ) {
+			parent.addEventListener( "unload", unloadHandler, false );
+		} else if ( parent.attachEvent ) {
+			parent.attachEvent( "onunload", unloadHandler );
+		}
+	}
+
+	/* Support tests
+	---------------------------------------------------------------------- */
+	documentIsHTML = !isXML( doc );
+
+	/* Attributes
+	---------------------------------------------------------------------- */
+
+	// Support: IE<8
+	// Verify that getAttribute really returns attributes and not properties
+	// (excepting IE8 booleans)
+	support.attributes = assert(function( div ) {
+		div.className = "i";
+		return !div.getAttribute("className");
+	});
+
+	/* getElement(s)By*
+	---------------------------------------------------------------------- */
+
+	// Check if getElementsByTagName("*") returns only elements
+	support.getElementsByTagName = assert(function( div ) {
+		div.appendChild( doc.createComment("") );
+		return !div.getElementsByTagName("*").length;
+	});
+
+	// Support: IE<9
+	support.getElementsByClassName = rnative.test( doc.getElementsByClassName );
+
+	// Support: IE<10
+	// Check if getElementById returns elements by name
+	// The broken getElementById methods don't pick up programatically-set names,
+	// so use a roundabout getElementsByName test
+	support.getById = assert(function( div ) {
+		docElem.appendChild( div ).id = expando;
+		return !doc.getElementsByName || !doc.getElementsByName( expando ).length;
+	});
+
+	// ID find and filter
+	if ( support.getById ) {
+		Expr.find["ID"] = function( id, context ) {
+			if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
+				var m = context.getElementById( id );
+				// Check parentNode to catch when Blackberry 4.6 returns
+				// nodes that are no longer in the document #6963
+				return m && m.parentNode ? [ m ] : [];
+			}
+		};
+		Expr.filter["ID"] = function( id ) {
+			var attrId = id.replace( runescape, funescape );
+			return function( elem ) {
+				return elem.getAttribute("id") === attrId;
+			};
+		};
+	} else {
+		// Support: IE6/7
+		// getElementById is not reliable as a find shortcut
+		delete Expr.find["ID"];
+
+		Expr.filter["ID"] =  function( id ) {
+			var attrId = id.replace( runescape, funescape );
+			return function( elem ) {
+				var node = typeof elem.getAttributeNode !== "undefined" && elem.getAttributeNode("id");
+				return node && node.value === attrId;
+			};
+		};
+	}
+
+	// Tag
+	Expr.find["TAG"] = support.getElementsByTagName ?
+		function( tag, context ) {
+			if ( typeof context.getElementsByTagName !== "undefined" ) {
+				return context.getElementsByTagName( tag );
+
+			// DocumentFragment nodes don't have gEBTN
+			} else if ( support.qsa ) {
+				return context.querySelectorAll( tag );
+			}
+		} :
+
+		function( tag, context ) {
+			var elem,
+				tmp = [],
+				i = 0,
+				// By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
+				results = context.getElementsByTagName( tag );
+
+			// Filter out possible comments
+			if ( tag === "*" ) {
+				while ( (elem = results[i++]) ) {
+					if ( elem.nodeType === 1 ) {
+						tmp.push( elem );
+					}
+				}
+
+				return tmp;
+			}
+			return results;
+		};
+
+	// Class
+	Expr.find["CLASS"] = support.getElementsByClassName && function( className, context ) {
+		if ( documentIsHTML ) {
+			return context.getElementsByClassName( className );
+		}
+	};
+
+	/* QSA/matchesSelector
+	---------------------------------------------------------------------- */
+
+	// QSA and matchesSelector support
+
+	// matchesSelector(:active) reports false when true (IE9/Opera 11.5)
+	rbuggyMatches = [];
+
+	// qSa(:focus) reports false when true (Chrome 21)
+	// We allow this because of a bug in IE8/9 that throws an error
+	// whenever `document.activeElement` is accessed on an iframe
+	// So, we allow :focus to pass through QSA all the time to avoid the IE error
+	// See http://bugs.jquery.com/ticket/13378
+	rbuggyQSA = [];
+
+	if ( (support.qsa = rnative.test( doc.querySelectorAll )) ) {
+		// Build QSA regex
+		// Regex strategy adopted from Diego Perini
+		assert(function( div ) {
+			// Select is set to empty string on purpose
+			// This is to test IE's treatment of not explicitly
+			// setting a boolean content attribute,
+			// since its presence should be enough
+			// http://bugs.jquery.com/ticket/12359
+			docElem.appendChild( div ).innerHTML = "<a id='" + expando + "'></a>" +
+				"<select id='" + expando + "-\f]' msallowcapture=''>" +
+				"<option selected=''></option></select>";
+
+			// Support: IE8, Opera 11-12.16
+			// Nothing should be selected when empty strings follow ^= or $= or *=
+			// The test attribute must be unknown in Opera but "safe" for WinRT
+			// http://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
+			if ( div.querySelectorAll("[msallowcapture^='']").length ) {
+				rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
+			}
+
+			// Support: IE8
+			// Boolean attributes and "value" are not treated correctly
+			if ( !div.querySelectorAll("[selected]").length ) {
+				rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
+			}
+
+			// Support: Chrome<29, Android<4.2+, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.7+
+			if ( !div.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
+				rbuggyQSA.push("~=");
+			}
+
+			// Webkit/Opera - :checked should return selected option elements
+			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+			// IE8 throws error here and will not see later tests
+			if ( !div.querySelectorAll(":checked").length ) {
+				rbuggyQSA.push(":checked");
+			}
+
+			// Support: Safari 8+, iOS 8+
+			// https://bugs.webkit.org/show_bug.cgi?id=136851
+			// In-page `selector#id sibing-combinator selector` fails
+			if ( !div.querySelectorAll( "a#" + expando + "+*" ).length ) {
+				rbuggyQSA.push(".#.+[+~]");
+			}
+		});
+
+		assert(function( div ) {
+			// Support: Windows 8 Native Apps
+			// The type and name attributes are restricted during .innerHTML assignment
+			var input = doc.createElement("input");
+			input.setAttribute( "type", "hidden" );
+			div.appendChild( input ).setAttribute( "name", "D" );
+
+			// Support: IE8
+			// Enforce case-sensitivity of name attribute
+			if ( div.querySelectorAll("[name=d]").length ) {
+				rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
+			}
+
+			// FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
+			// IE8 throws error here and will not see later tests
+			if ( !div.querySelectorAll(":enabled").length ) {
+				rbuggyQSA.push( ":enabled", ":disabled" );
+			}
+
+			// Opera 10-11 does not throw on post-comma invalid pseudos
+			div.querySelectorAll("*,:x");
+			rbuggyQSA.push(",.*:");
+		});
+	}
+
+	if ( (support.matchesSelector = rnative.test( (matches = docElem.matches ||
+		docElem.webkitMatchesSelector ||
+		docElem.mozMatchesSelector ||
+		docElem.oMatchesSelector ||
+		docElem.msMatchesSelector) )) ) {
+
+		assert(function( div ) {
+			// Check to see if it's possible to do matchesSelector
+			// on a disconnected node (IE 9)
+			support.disconnectedMatch = matches.call( div, "div" );
+
+			// This should fail with an exception
+			// Gecko does not error, returns false instead
+			matches.call( div, "[s!='']:x" );
+			rbuggyMatches.push( "!=", pseudos );
+		});
+	}
+
+	rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join("|") );
+	rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join("|") );
+
+	/* Contains
+	---------------------------------------------------------------------- */
+	hasCompare = rnative.test( docElem.compareDocumentPosition );
+
+	// Element contains another
+	// Purposefully does not implement inclusive descendent
+	// As in, an element does not contain itself
+	contains = hasCompare || rnative.test( docElem.contains ) ?
+		function( a, b ) {
+			var adown = a.nodeType === 9 ? a.documentElement : a,
+				bup = b && b.parentNode;
+			return a === bup || !!( bup && bup.nodeType === 1 && (
+				adown.contains ?
+					adown.contains( bup ) :
+					a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
+			));
+		} :
+		function( a, b ) {
+			if ( b ) {
+				while ( (b = b.parentNode) ) {
+					if ( b === a ) {
+						return true;
+					}
+				}
+			}
+			return false;
+		};
+
+	/* Sorting
+	---------------------------------------------------------------------- */
+
+	// Document order sorting
+	sortOrder = hasCompare ?
+	function( a, b ) {
+
+		// Flag for duplicate removal
+		if ( a === b ) {
+			hasDuplicate = true;
+			return 0;
+		}
+
+		// Sort on method existence if only one input has compareDocumentPosition
+		var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
+		if ( compare ) {
+			return compare;
+		}
+
+		// Calculate position if both inputs belong to the same document
+		compare = ( a.ownerDocument || a ) === ( b.ownerDocument || b ) ?
+			a.compareDocumentPosition( b ) :
+
+			// Otherwise we know they are disconnected
+			1;
+
+		// Disconnected nodes
+		if ( compare & 1 ||
+			(!support.sortDetached && b.compareDocumentPosition( a ) === compare) ) {
+
+			// Choose the first element that is related to our preferred document
+			if ( a === doc || a.ownerDocument === preferredDoc && contains(preferredDoc, a) ) {
+				return -1;
+			}
+			if ( b === doc || b.ownerDocument === preferredDoc && contains(preferredDoc, b) ) {
+				return 1;
+			}
+
+			// Maintain original order
+			return sortInput ?
+				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+				0;
+		}
+
+		return compare & 4 ? -1 : 1;
+	} :
+	function( a, b ) {
+		// Exit early if the nodes are identical
+		if ( a === b ) {
+			hasDuplicate = true;
+			return 0;
+		}
+
+		var cur,
+			i = 0,
+			aup = a.parentNode,
+			bup = b.parentNode,
+			ap = [ a ],
+			bp = [ b ];
+
+		// Parentless nodes are either documents or disconnected
+		if ( !aup || !bup ) {
+			return a === doc ? -1 :
+				b === doc ? 1 :
+				aup ? -1 :
+				bup ? 1 :
+				sortInput ?
+				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+				0;
+
+		// If the nodes are siblings, we can do a quick check
+		} else if ( aup === bup ) {
+			return siblingCheck( a, b );
+		}
+
+		// Otherwise we need full lists of their ancestors for comparison
+		cur = a;
+		while ( (cur = cur.parentNode) ) {
+			ap.unshift( cur );
+		}
+		cur = b;
+		while ( (cur = cur.parentNode) ) {
+			bp.unshift( cur );
+		}
+
+		// Walk down the tree looking for a discrepancy
+		while ( ap[i] === bp[i] ) {
+			i++;
+		}
+
+		return i ?
+			// Do a sibling check if the nodes have a common ancestor
+			siblingCheck( ap[i], bp[i] ) :
+
+			// Otherwise nodes in our document sort first
+			ap[i] === preferredDoc ? -1 :
+			bp[i] === preferredDoc ? 1 :
+			0;
+	};
+
+	return doc;
+};
+
+Sizzle.matches = function( expr, elements ) {
+	return Sizzle( expr, null, null, elements );
+};
+
+Sizzle.matchesSelector = function( elem, expr ) {
+	// Set document vars if needed
+	if ( ( elem.ownerDocument || elem ) !== document ) {
+		setDocument( elem );
+	}
+
+	// Make sure that attribute selectors are quoted
+	expr = expr.replace( rattributeQuotes, "='$1']" );
+
+	if ( support.matchesSelector && documentIsHTML &&
+		( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
+		( !rbuggyQSA     || !rbuggyQSA.test( expr ) ) ) {
+
+		try {
+			var ret = matches.call( elem, expr );
+
+			// IE 9's matchesSelector returns false on disconnected nodes
+			if ( ret || support.disconnectedMatch ||
+					// As well, disconnected nodes are said to be in a document
+					// fragment in IE 9
+					elem.document && elem.document.nodeType !== 11 ) {
+				return ret;
+			}
+		} catch (e) {}
+	}
+
+	return Sizzle( expr, document, null, [ elem ] ).length > 0;
+};
+
+Sizzle.contains = function( context, elem ) {
+	// Set document vars if needed
+	if ( ( context.ownerDocument || context ) !== document ) {
+		setDocument( context );
+	}
+	return contains( context, elem );
+};
+
+Sizzle.attr = function( elem, name ) {
+	// Set document vars if needed
+	if ( ( elem.ownerDocument || elem ) !== document ) {
+		setDocument( elem );
+	}
+
+	var fn = Expr.attrHandle[ name.toLowerCase() ],
+		// Don't get fooled by Object.prototype properties (jQuery #13807)
+		val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
+			fn( elem, name, !documentIsHTML ) :
+			undefined;
+
+	return val !== undefined ?
+		val :
+		support.attributes || !documentIsHTML ?
+			elem.getAttribute( name ) :
+			(val = elem.getAttributeNode(name)) && val.specified ?
+				val.value :
+				null;
+};
+
+Sizzle.error = function( msg ) {
+	throw new Error( "Syntax error, unrecognized expression: " + msg );
+};
+
+/**
+ * Document sorting and removing duplicates
+ * @param {ArrayLike} results
+ */
+Sizzle.uniqueSort = function( results ) {
+	var elem,
+		duplicates = [],
+		j = 0,
+		i = 0;
+
+	// Unless we *know* we can detect duplicates, assume their presence
+	hasDuplicate = !support.detectDuplicates;
+	sortInput = !support.sortStable && results.slice( 0 );
+	results.sort( sortOrder );
+
+	if ( hasDuplicate ) {
+		while ( (elem = results[i++]) ) {
+			if ( elem === results[ i ] ) {
+				j = duplicates.push( i );
+			}
+		}
+		while ( j-- ) {
+			results.splice( duplicates[ j ], 1 );
+		}
+	}
+
+	// Clear input after sorting to release objects
+	// See https://github.com/jquery/sizzle/pull/225
+	sortInput = null;
+
+	return results;
+};
+
+/**
+ * Utility function for retrieving the text value of an array of DOM nodes
+ * @param {Array|Element} elem
+ */
+getText = Sizzle.getText = function( elem ) {
+	var node,
+		ret = "",
+		i = 0,
+		nodeType = elem.nodeType;
+
+	if ( !nodeType ) {
+		// If no nodeType, this is expected to be an array
+		while ( (node = elem[i++]) ) {
+			// Do not traverse comment nodes
+			ret += getText( node );
+		}
+	} else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
+		// Use textContent for elements
+		// innerText usage removed for consistency of new lines (jQuery #11153)
+		if ( typeof elem.textContent === "string" ) {
+			return elem.textContent;
+		} else {
+			// Traverse its children
+			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+				ret += getText( elem );
+			}
+		}
+	} else if ( nodeType === 3 || nodeType === 4 ) {
+		return elem.nodeValue;
+	}
+	// Do not include comment or processing instruction nodes
+
+	return ret;
+};
+
+Expr = Sizzle.selectors = {
+
+	// Can be adjusted by the user
+	cacheLength: 50,
+
+	createPseudo: markFunction,
+
+	match: matchExpr,
+
+	attrHandle: {},
+
+	find: {},
+
+	relative: {
+		">": { dir: "parentNode", first: true },
+		" ": { dir: "parentNode" },
+		"+": { dir: "previousSibling", first: true },
+		"~": { dir: "previousSibling" }
+	},
+
+	preFilter: {
+		"ATTR": function( match ) {
+			match[1] = match[1].replace( runescape, funescape );
+
+			// Move the given value to match[3] whether quoted or unquoted
+			match[3] = ( match[3] || match[4] || match[5] || "" ).replace( runescape, funescape );
+
+			if ( match[2] === "~=" ) {
+				match[3] = " " + match[3] + " ";
+			}
+
+			return match.slice( 0, 4 );
+		},
+
+		"CHILD": function( match ) {
+			/* matches from matchExpr["CHILD"]
+				1 type (only|nth|...)
+				2 what (child|of-type)
+				3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
+				4 xn-component of xn+y argument ([+-]?\d*n|)
+				5 sign of xn-component
+				6 x of xn-component
+				7 sign of y-component
+				8 y of y-component
+			*/
+			match[1] = match[1].toLowerCase();
+
+			if ( match[1].slice( 0, 3 ) === "nth" ) {
+				// nth-* requires argument
+				if ( !match[3] ) {
+					Sizzle.error( match[0] );
+				}
+
+				// numeric x and y parameters for Expr.filter.CHILD
+				// remember that false/true cast respectively to 0/1
+				match[4] = +( match[4] ? match[5] + (match[6] || 1) : 2 * ( match[3] === "even" || match[3] === "odd" ) );
+				match[5] = +( ( match[7] + match[8] ) || match[3] === "odd" );
+
+			// other types prohibit arguments
+			} else if ( match[3] ) {
+				Sizzle.error( match[0] );
+			}
+
+			return match;
+		},
+
+		"PSEUDO": function( match ) {
+			var excess,
+				unquoted = !match[6] && match[2];
+
+			if ( matchExpr["CHILD"].test( match[0] ) ) {
+				return null;
+			}
+
+			// Accept quoted arguments as-is
+			if ( match[3] ) {
+				match[2] = match[4] || match[5] || "";
+
+			// Strip excess characters from unquoted arguments
+			} else if ( unquoted && rpseudo.test( unquoted ) &&
+				// Get excess from tokenize (recursively)
+				(excess = tokenize( unquoted, true )) &&
+				// advance to the next closing parenthesis
+				(excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length) ) {
+
+				// excess is a negative index
+				match[0] = match[0].slice( 0, excess );
+				match[2] = unquoted.slice( 0, excess );
+			}
+
+			// Return only captures needed by the pseudo filter method (type and argument)
+			return match.slice( 0, 3 );
+		}
+	},
+
+	filter: {
+
+		"TAG": function( nodeNameSelector ) {
+			var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
+			return nodeNameSelector === "*" ?
+				function() { return true; } :
+				function( elem ) {
+					return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
+				};
+		},
+
+		"CLASS": function( className ) {
+			var pattern = classCache[ className + " " ];
+
+			return pattern ||
+				(pattern = new RegExp( "(^|" + whitespace + ")" + className + "(" + whitespace + "|$)" )) &&
+				classCache( className, function( elem ) {
+					return pattern.test( typeof elem.className === "string" && elem.className || typeof elem.getAttribute !== "undefined" && elem.getAttribute("class") || "" );
+				});
+		},
+
+		"ATTR": function( name, operator, check ) {
+			return function( elem ) {
+				var result = Sizzle.attr( elem, name );
+
+				if ( result == null ) {
+					return operator === "!=";
+				}
+				if ( !operator ) {
+					return true;
+				}
+
+				result += "";
+
+				return operator === "=" ? result === check :
+					operator === "!=" ? result !== check :
+					operator === "^=" ? check && result.indexOf( check ) === 0 :
+					operator === "*=" ? check && result.indexOf( check ) > -1 :
+					operator === "$=" ? check && result.slice( -check.length ) === check :
+					operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
+					operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
+					false;
+			};
+		},
+
+		"CHILD": function( type, what, argument, first, last ) {
+			var simple = type.slice( 0, 3 ) !== "nth",
+				forward = type.slice( -4 ) !== "last",
+				ofType = what === "of-type";
+
+			return first === 1 && last === 0 ?
+
+				// Shortcut for :nth-*(n)
+				function( elem ) {
+					return !!elem.parentNode;
+				} :
+
+				function( elem, context, xml ) {
+					var cache, outerCache, node, diff, nodeIndex, start,
+						dir = simple !== forward ? "nextSibling" : "previousSibling",
+						parent = elem.parentNode,
+						name = ofType && elem.nodeName.toLowerCase(),
+						useCache = !xml && !ofType;
+
+					if ( parent ) {
+
+						// :(first|last|only)-(child|of-type)
+						if ( simple ) {
+							while ( dir ) {
+								node = elem;
+								while ( (node = node[ dir ]) ) {
+									if ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) {
+										return false;
+									}
+								}
+								// Reverse direction for :only-* (if we haven't yet done so)
+								start = dir = type === "only" && !start && "nextSibling";
+							}
+							return true;
+						}
+
+						start = [ forward ? parent.firstChild : parent.lastChild ];
+
+						// non-xml :nth-child(...) stores cache data on `parent`
+						if ( forward && useCache ) {
+							// Seek `elem` from a previously-cached index
+							outerCache = parent[ expando ] || (parent[ expando ] = {});
+							cache = outerCache[ type ] || [];
+							nodeIndex = cache[0] === dirruns && cache[1];
+							diff = cache[0] === dirruns && cache[2];
+							node = nodeIndex && parent.childNodes[ nodeIndex ];
+
+							while ( (node = ++nodeIndex && node && node[ dir ] ||
+
+								// Fallback to seeking `elem` from the start
+								(diff = nodeIndex = 0) || start.pop()) ) {
+
+								// When found, cache indexes on `parent` and break
+								if ( node.nodeType === 1 && ++diff && node === elem ) {
+									outerCache[ type ] = [ dirruns, nodeIndex, diff ];
+									break;
+								}
+							}
+
+						// Use previously-cached element index if available
+						} else if ( useCache && (cache = (elem[ expando ] || (elem[ expando ] = {}))[ type ]) && cache[0] === dirruns ) {
+							diff = cache[1];
+
+						// xml :nth-child(...) or :nth-last-child(...) or :nth(-last)?-of-type(...)
+						} else {
+							// Use the same loop as above to seek `elem` from the start
+							while ( (node = ++nodeIndex && node && node[ dir ] ||
+								(diff = nodeIndex = 0) || start.pop()) ) {
+
+								if ( ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) && ++diff ) {
+									// Cache the index of each encountered element
+									if ( useCache ) {
+										(node[ expando ] || (node[ expando ] = {}))[ type ] = [ dirruns, diff ];
+									}
+
+									if ( node === elem ) {
+										break;
+									}
+								}
+							}
+						}
+
+						// Incorporate the offset, then check against cycle size
+						diff -= last;
+						return diff === first || ( diff % first === 0 && diff / first >= 0 );
+					}
+				};
+		},
+
+		"PSEUDO": function( pseudo, argument ) {
+			// pseudo-class names are case-insensitive
+			// http://www.w3.org/TR/selectors/#pseudo-classes
+			// Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
+			// Remember that setFilters inherits from pseudos
+			var args,
+				fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
+					Sizzle.error( "unsupported pseudo: " + pseudo );
+
+			// The user may use createPseudo to indicate that
+			// arguments are needed to create the filter function
+			// just as Sizzle does
+			if ( fn[ expando ] ) {
+				return fn( argument );
+			}
+
+			// But maintain support for old signatures
+			if ( fn.length > 1 ) {
+				args = [ pseudo, pseudo, "", argument ];
+				return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
+					markFunction(function( seed, matches ) {
+						var idx,
+							matched = fn( seed, argument ),
+							i = matched.length;
+						while ( i-- ) {
+							idx = indexOf( seed, matched[i] );
+							seed[ idx ] = !( matches[ idx ] = matched[i] );
+						}
+					}) :
+					function( elem ) {
+						return fn( elem, 0, args );
+					};
+			}
+
+			return fn;
+		}
+	},
+
+	pseudos: {
+		// Potentially complex pseudos
+		"not": markFunction(function( selector ) {
+			// Trim the selector passed to compile
+			// to avoid treating leading and trailing
+			// spaces as combinators
+			var input = [],
+				results = [],
+				matcher = compile( selector.replace( rtrim, "$1" ) );
+
+			return matcher[ expando ] ?
+				markFunction(function( seed, matches, context, xml ) {
+					var elem,
+						unmatched = matcher( seed, null, xml, [] ),
+						i = seed.length;
+
+					// Match elements unmatched by `matcher`
+					while ( i-- ) {
+						if ( (elem = unmatched[i]) ) {
+							seed[i] = !(matches[i] = elem);
+						}
+					}
+				}) :
+				function( elem, context, xml ) {
+					input[0] = elem;
+					matcher( input, null, xml, results );
+					// Don't keep the element (issue #299)
+					input[0] = null;
+					return !results.pop();
+				};
+		}),
+
+		"has": markFunction(function( selector ) {
+			return function( elem ) {
+				return Sizzle( selector, elem ).length > 0;
+			};
+		}),
+
+		"contains": markFunction(function( text ) {
+			text = text.replace( runescape, funescape );
+			return function( elem ) {
+				return ( elem.textContent || elem.innerText || getText( elem ) ).indexOf( text ) > -1;
+			};
+		}),
+
+		// "Whether an element is represented by a :lang() selector
+		// is based solely on the element's language value
+		// being equal to the identifier C,
+		// or beginning with the identifier C immediately followed by "-".
+		// The matching of C against the element's language value is performed case-insensitively.
+		// The identifier C does not have to be a valid language name."
+		// http://www.w3.org/TR/selectors/#lang-pseudo
+		"lang": markFunction( function( lang ) {
+			// lang value must be a valid identifier
+			if ( !ridentifier.test(lang || "") ) {
+				Sizzle.error( "unsupported lang: " + lang );
+			}
+			lang = lang.replace( runescape, funescape ).toLowerCase();
+			return function( elem ) {
+				var elemLang;
+				do {
+					if ( (elemLang = documentIsHTML ?
+						elem.lang :
+						elem.getAttribute("xml:lang") || elem.getAttribute("lang")) ) {
+
+						elemLang = elemLang.toLowerCase();
+						return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
+					}
+				} while ( (elem = elem.parentNode) && elem.nodeType === 1 );
+				return false;
+			};
+		}),
+
+		// Miscellaneous
+		"target": function( elem ) {
+			var hash = window.location && window.location.hash;
+			return hash && hash.slice( 1 ) === elem.id;
+		},
+
+		"root": function( elem ) {
+			return elem === docElem;
+		},
+
+		"focus": function( elem ) {
+			return elem === document.activeElement && (!document.hasFocus || document.hasFocus()) && !!(elem.type || elem.href || ~elem.tabIndex);
+		},
+
+		// Boolean properties
+		"enabled": function( elem ) {
+			return elem.disabled === false;
+		},
+
+		"disabled": function( elem ) {
+			return elem.disabled === true;
+		},
+
+		"checked": function( elem ) {
+			// In CSS3, :checked should return both checked and selected elements
+			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+			var nodeName = elem.nodeName.toLowerCase();
+			return (nodeName === "input" && !!elem.checked) || (nodeName === "option" && !!elem.selected);
+		},
+
+		"selected": function( elem ) {
+			// Accessing this property makes selected-by-default
+			// options in Safari work properly
+			if ( elem.parentNode ) {
+				elem.parentNode.selectedIndex;
+			}
+
+			return elem.selected === true;
+		},
+
+		// Contents
+		"empty": function( elem ) {
+			// http://www.w3.org/TR/selectors/#empty-pseudo
+			// :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
+			//   but not by others (comment: 8; processing instruction: 7; etc.)
+			// nodeType < 6 works because attributes (2) do not appear as children
+			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+				if ( elem.nodeType < 6 ) {
+					return false;
+				}
+			}
+			return true;
+		},
+
+		"parent": function( elem ) {
+			return !Expr.pseudos["empty"]( elem );
+		},
+
+		// Element/input types
+		"header": function( elem ) {
+			return rheader.test( elem.nodeName );
+		},
+
+		"input": function( elem ) {
+			return rinputs.test( elem.nodeName );
+		},
+
+		"button": function( elem ) {
+			var name = elem.nodeName.toLowerCase();
+			return name === "input" && elem.type === "button" || name === "button";
+		},
+
+		"text": function( elem ) {
+			var attr;
+			return elem.nodeName.toLowerCase() === "input" &&
+				elem.type === "text" &&
+
+				// Support: IE<8
+				// New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
+				( (attr = elem.getAttribute("type")) == null || attr.toLowerCase() === "text" );
+		},
+
+		// Position-in-collection
+		"first": createPositionalPseudo(function() {
+			return [ 0 ];
+		}),
+
+		"last": createPositionalPseudo(function( matchIndexes, length ) {
+			return [ length - 1 ];
+		}),
+
+		"eq": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			return [ argument < 0 ? argument + length : argument ];
+		}),
+
+		"even": createPositionalPseudo(function( matchIndexes, length ) {
+			var i = 0;
+			for ( ; i < length; i += 2 ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"odd": createPositionalPseudo(function( matchIndexes, length ) {
+			var i = 1;
+			for ( ; i < length; i += 2 ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"lt": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			var i = argument < 0 ? argument + length : argument;
+			for ( ; --i >= 0; ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"gt": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			var i = argument < 0 ? argument + length : argument;
+			for ( ; ++i < length; ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		})
+	}
+};
+
+Expr.pseudos["nth"] = Expr.pseudos["eq"];
+
+// Add button/input type pseudos
+for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
+	Expr.pseudos[ i ] = createInputPseudo( i );
+}
+for ( i in { submit: true, reset: true } ) {
+	Expr.pseudos[ i ] = createButtonPseudo( i );
+}
+
+// Easy API for creating new setFilters
+function setFilters() {}
+setFilters.prototype = Expr.filters = Expr.pseudos;
+Expr.setFilters = new setFilters();
+
+tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
+	var matched, match, tokens, type,
+		soFar, groups, preFilters,
+		cached = tokenCache[ selector + " " ];
+
+	if ( cached ) {
+		return parseOnly ? 0 : cached.slice( 0 );
+	}
+
+	soFar = selector;
+	groups = [];
+	preFilters = Expr.preFilter;
+
+	while ( soFar ) {
+
+		// Comma and first run
+		if ( !matched || (match = rcomma.exec( soFar )) ) {
+			if ( match ) {
+				// Don't consume trailing commas as valid
+				soFar = soFar.slice( match[0].length ) || soFar;
+			}
+			groups.push( (tokens = []) );
+		}
+
+		matched = false;
+
+		// Combinators
+		if ( (match = rcombinators.exec( soFar )) ) {
+			matched = match.shift();
+			tokens.push({
+				value: matched,
+				// Cast descendant combinators to space
+				type: match[0].replace( rtrim, " " )
+			});
+			soFar = soFar.slice( matched.length );
+		}
+
+		// Filters
+		for ( type in Expr.filter ) {
+			if ( (match = matchExpr[ type ].exec( soFar )) && (!preFilters[ type ] ||
+				(match = preFilters[ type ]( match ))) ) {
+				matched = match.shift();
+				tokens.push({
+					value: matched,
+					type: type,
+					matches: match
+				});
+				soFar = soFar.slice( matched.length );
+			}
+		}
+
+		if ( !matched ) {
+			break;
+		}
+	}
+
+	// Return the length of the invalid excess
+	// if we're just parsing
+	// Otherwise, throw an error or return tokens
+	return parseOnly ?
+		soFar.length :
+		soFar ?
+			Sizzle.error( selector ) :
+			// Cache the tokens
+			tokenCache( selector, groups ).slice( 0 );
+};
+
+function toSelector( tokens ) {
+	var i = 0,
+		len = tokens.length,
+		selector = "";
+	for ( ; i < len; i++ ) {
+		selector += tokens[i].value;
+	}
+	return selector;
+}
+
+function addCombinator( matcher, combinator, base ) {
+	var dir = combinator.dir,
+		checkNonElements = base && dir === "parentNode",
+		doneName = done++;
+
+	return combinator.first ?
+		// Check against closest ancestor/preceding element
+		function( elem, context, xml ) {
+			while ( (elem = elem[ dir ]) ) {
+				if ( elem.nodeType === 1 || checkNonElements ) {
+					return matcher( elem, context, xml );
+				}
+			}
+		} :
+
+		// Check against all ancestor/preceding elements
+		function( elem, context, xml ) {
+			var oldCache, outerCache,
+				newCache = [ dirruns, doneName ];
+
+			// We can't set arbitrary data on XML nodes, so they don't benefit from dir caching
+			if ( xml ) {
+				while ( (elem = elem[ dir ]) ) {
+					if ( elem.nodeType === 1 || checkNonElements ) {
+						if ( matcher( elem, context, xml ) ) {
+							return true;
+						}
+					}
+				}
+			} else {
+				while ( (elem = elem[ dir ]) ) {
+					if ( elem.nodeType === 1 || checkNonElements ) {
+						outerCache = elem[ expando ] || (elem[ expando ] = {});
+						if ( (oldCache = outerCache[ dir ]) &&
+							oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
+
+							// Assign to newCache so results back-propagate to previous elements
+							return (newCache[ 2 ] = oldCache[ 2 ]);
+						} else {
+							// Reuse newcache so results back-propagate to previous elements
+							outerCache[ dir ] = newCache;
+
+							// A match means we're done; a fail means we have to keep checking
+							if ( (newCache[ 2 ] = matcher( elem, context, xml )) ) {
+								return true;
+							}
+						}
+					}
+				}
+			}
+		};
+}
+
+function elementMatcher( matchers ) {
+	return matchers.length > 1 ?
+		function( elem, context, xml ) {
+			var i = matchers.length;
+			while ( i-- ) {
+				if ( !matchers[i]( elem, context, xml ) ) {
+					return false;
+				}
+			}
+			return true;
+		} :
+		matchers[0];
+}
+
+function multipleContexts( selector, contexts, results ) {
+	var i = 0,
+		len = contexts.length;
+	for ( ; i < len; i++ ) {
+		Sizzle( selector, contexts[i], results );
+	}
+	return results;
+}
+
+function condense( unmatched, map, filter, context, xml ) {
+	var elem,
+		newUnmatched = [],
+		i = 0,
+		len = unmatched.length,
+		mapped = map != null;
+
+	for ( ; i < len; i++ ) {
+		if ( (elem = unmatched[i]) ) {
+			if ( !filter || filter( elem, context, xml ) ) {
+				newUnmatched.push( elem );
+				if ( mapped ) {
+					map.push( i );
+				}
+			}
+		}
+	}
+
+	return newUnmatched;
+}
+
+function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
+	if ( postFilter && !postFilter[ expando ] ) {
+		postFilter = setMatcher( postFilter );
+	}
+	if ( postFinder && !postFinder[ expando ] ) {
+		postFinder = setMatcher( postFinder, postSelector );
+	}
+	return markFunction(function( seed, results, context, xml ) {
+		var temp, i, elem,
+			preMap = [],
+			postMap = [],
+			preexisting = results.length,
+
+			// Get initial elements from seed or context
+			elems = seed || multipleContexts( selector || "*", context.nodeType ? [ context ] : context, [] ),
+
+			// Prefilter to get matcher input, preserving a map for seed-results synchronization
+			matcherIn = preFilter && ( seed || !selector ) ?
+				condense( elems, preMap, preFilter, context, xml ) :
+				elems,
+
+			matcherOut = matcher ?
+				// If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
+				postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
+
+					// ...intermediate processing is necessary
+					[] :
+
+					// ...otherwise use results directly
+					results :
+				matcherIn;
+
+		// Find primary matches
+		if ( matcher ) {
+			matcher( matcherIn, matcherOut, context, xml );
+		}
+
+		// Apply postFilter
+		if ( postFilter ) {
+			temp = condense( matcherOut, postMap );
+			postFilter( temp, [], context, xml );
+
+			// Un-match failing elements by moving them back to matcherIn
+			i = temp.length;
+			while ( i-- ) {
+				if ( (elem = temp[i]) ) {
+					matcherOut[ postMap[i] ] = !(matcherIn[ postMap[i] ] = elem);
+				}
+			}
+		}
+
+		if ( seed ) {
+			if ( postFinder || preFilter ) {
+				if ( postFinder ) {
+					// Get the final matcherOut by condensing this intermediate into postFinder contexts
+					temp = [];
+					i = matcherOut.length;
+					while ( i-- ) {
+						if ( (elem = matcherOut[i]) ) {
+							// Restore matcherIn since elem is not yet a final match
+							temp.push( (matcherIn[i] = elem) );
+						}
+					}
+					postFinder( null, (matcherOut = []), temp, xml );
+				}
+
+				// Move matched elements from seed to results to keep them synchronized
+				i = matcherOut.length;
+				while ( i-- ) {
+					if ( (elem = matcherOut[i]) &&
+						(temp = postFinder ? indexOf( seed, elem ) : preMap[i]) > -1 ) {
+
+						seed[temp] = !(results[temp] = elem);
+					}
+				}
+			}
+
+		// Add elements to results, through postFinder if defined
+		} else {
+			matcherOut = condense(
+				matcherOut === results ?
+					matcherOut.splice( preexisting, matcherOut.length ) :
+					matcherOut
+			);
+			if ( postFinder ) {
+				postFinder( null, results, matcherOut, xml );
+			} else {
+				push.apply( results, matcherOut );
+			}
+		}
+	});
+}
+
+function matcherFromTokens( tokens ) {
+	var checkContext, matcher, j,
+		len = tokens.length,
+		leadingRelative = Expr.relative[ tokens[0].type ],
+		implicitRelative = leadingRelative || Expr.relative[" "],
+		i = leadingRelative ? 1 : 0,
+
+		// The foundational matcher ensures that elements are reachable from top-level context(s)
+		matchContext = addCombinator( function( elem ) {
+			return elem === checkContext;
+		}, implicitRelative, true ),
+		matchAnyContext = addCombinator( function( elem ) {
+			return indexOf( checkContext, elem ) > -1;
+		}, implicitRelative, true ),
+		matchers = [ function( elem, context, xml ) {
+			var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
+				(checkContext = context).nodeType ?
+					matchContext( elem, context, xml ) :
+					matchAnyContext( elem, context, xml ) );
+			// Avoid hanging onto element (issue #299)
+			checkContext = null;
+			return ret;
+		} ];
+
+	for ( ; i < len; i++ ) {
+		if ( (matcher = Expr.relative[ tokens[i].type ]) ) {
+			matchers = [ addCombinator(elementMatcher( matchers ), matcher) ];
+		} else {
+			matcher = Expr.filter[ tokens[i].type ].apply( null, tokens[i].matches );
+
+			// Return special upon seeing a positional matcher
+			if ( matcher[ expando ] ) {
+				// Find the next relative operator (if any) for proper handling
+				j = ++i;
+				for ( ; j < len; j++ ) {
+					if ( Expr.relative[ tokens[j].type ] ) {
+						break;
+					}
+				}
+				return setMatcher(
+					i > 1 && elementMatcher( matchers ),
+					i > 1 && toSelector(
+						// If the preceding token was a descendant combinator, insert an implicit any-element `*`
+						tokens.slice( 0, i - 1 ).concat({ value: tokens[ i - 2 ].type === " " ? "*" : "" })
+					).replace( rtrim, "$1" ),
+					matcher,
+					i < j && matcherFromTokens( tokens.slice( i, j ) ),
+					j < len && matcherFromTokens( (tokens = tokens.slice( j )) ),
+					j < len && toSelector( tokens )
+				);
+			}
+			matchers.push( matcher );
+		}
+	}
+
+	return elementMatcher( matchers );
+}
+
+function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
+	var bySet = setMatchers.length > 0,
+		byElement = elementMatchers.length > 0,
+		superMatcher = function( seed, context, xml, results, outermost ) {
+			var elem, j, matcher,
+				matchedCount = 0,
+				i = "0",
+				unmatched = seed && [],
+				setMatched = [],
+				contextBackup = outermostContext,
+				// We must always have either seed elements or outermost context
+				elems = seed || byElement && Expr.find["TAG"]( "*", outermost ),
+				// Use integer dirruns iff this is the outermost matcher
+				dirrunsUnique = (dirruns += contextBackup == null ? 1 : Math.random() || 0.1),
+				len = elems.length;
+
+			if ( outermost ) {
+				outermostContext = context !== document && context;
+			}
+
+			// Add elements passing elementMatchers directly to results
+			// Keep `i` a string if there are no elements so `matchedCount` will be "00" below
+			// Support: IE<9, Safari
+			// Tolerate NodeList properties (IE: "length"; Safari: <number>) matching elements by id
+			for ( ; i !== len && (elem = elems[i]) != null; i++ ) {
+				if ( byElement && elem ) {
+					j = 0;
+					while ( (matcher = elementMatchers[j++]) ) {
+						if ( matcher( elem, context, xml ) ) {
+							results.push( elem );
+							break;
+						}
+					}
+					if ( outermost ) {
+						dirruns = dirrunsUnique;
+					}
+				}
+
+				// Track unmatched elements for set filters
+				if ( bySet ) {
+					// They will have gone through all possible matchers
+					if ( (elem = !matcher && elem) ) {
+						matchedCount--;
+					}
+
+					// Lengthen the array for every element, matched or not
+					if ( seed ) {
+						unmatched.push( elem );
+					}
+				}
+			}
+
+			// Apply set filters to unmatched elements
+			matchedCount += i;
+			if ( bySet && i !== matchedCount ) {
+				j = 0;
+				while ( (matcher = setMatchers[j++]) ) {
+					matcher( unmatched, setMatched, context, xml );
+				}
+
+				if ( seed ) {
+					// Reintegrate element matches to eliminate the need for sorting
+					if ( matchedCount > 0 ) {
+						while ( i-- ) {
+							if ( !(unmatched[i] || setMatched[i]) ) {
+								setMatched[i] = pop.call( results );
+							}
+						}
+					}
+
+					// Discard index placeholder values to get only actual matches
+					setMatched = condense( setMatched );
+				}
+
+				// Add matches to results
+				push.apply( results, setMatched );
+
+				// Seedless set matches succeeding multiple successful matchers stipulate sorting
+				if ( outermost && !seed && setMatched.length > 0 &&
+					( matchedCount + setMatchers.length ) > 1 ) {
+
+					Sizzle.uniqueSort( results );
+				}
+			}
+
+			// Override manipulation of globals by nested matchers
+			if ( outermost ) {
+				dirruns = dirrunsUnique;
+				outermostContext = contextBackup;
+			}
+
+			return unmatched;
+		};
+
+	return bySet ?
+		markFunction( superMatcher ) :
+		superMatcher;
+}
+
+compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
+	var i,
+		setMatchers = [],
+		elementMatchers = [],
+		cached = compilerCache[ selector + " " ];
+
+	if ( !cached ) {
+		// Generate a function of recursive functions that can be used to check each element
+		if ( !match ) {
+			match = tokenize( selector );
+		}
+		i = match.length;
+		while ( i-- ) {
+			cached = matcherFromTokens( match[i] );
+			if ( cached[ expando ] ) {
+				setMatchers.push( cached );
+			} else {
+				elementMatchers.push( cached );
+			}
+		}
+
+		// Cache the compiled function
+		cached = compilerCache( selector, matcherFromGroupMatchers( elementMatchers, setMatchers ) );
+
+		// Save selector and tokenization
+		cached.selector = selector;
+	}
+	return cached;
+};
+
+/**
+ * A low-level selection function that works with Sizzle's compiled
+ *  selector functions
+ * @param {String|Function} selector A selector or a pre-compiled
+ *  selector function built with Sizzle.compile
+ * @param {Element} context
+ * @param {Array} [results]
+ * @param {Array} [seed] A set of elements to match against
+ */
+select = Sizzle.select = function( selector, context, results, seed ) {
+	var i, tokens, token, type, find,
+		compiled = typeof selector === "function" && selector,
+		match = !seed && tokenize( (selector = compiled.selector || selector) );
+
+	results = results || [];
+
+	// Try to minimize operations if there is no seed and only one group
+	if ( match.length === 1 ) {
+
+		// Take a shortcut and set the context if the root selector is an ID
+		tokens = match[0] = match[0].slice( 0 );
+		if ( tokens.length > 2 && (token = tokens[0]).type === "ID" &&
+				support.getById && context.nodeType === 9 && documentIsHTML &&
+				Expr.relative[ tokens[1].type ] ) {
+
+			context = ( Expr.find["ID"]( token.matches[0].replace(runescape, funescape), context ) || [] )[0];
+			if ( !context ) {
+				return results;
+
+			// Precompiled matchers will still verify ancestry, so step up a level
+			} else if ( compiled ) {
+				context = context.parentNode;
+			}
+
+			selector = selector.slice( tokens.shift().value.length );
+		}
+
+		// Fetch a seed set for right-to-left matching
+		i = matchExpr["needsContext"].test( selector ) ? 0 : tokens.length;
+		while ( i-- ) {
+			token = tokens[i];
+
+			// Abort if we hit a combinator
+			if ( Expr.relative[ (type = token.type) ] ) {
+				break;
+			}
+			if ( (find = Expr.find[ type ]) ) {
+				// Search, expanding context for leading sibling combinators
+				if ( (seed = find(
+					token.matches[0].replace( runescape, funescape ),
+					rsibling.test( tokens[0].type ) && testContext( context.parentNode ) || context
+				)) ) {
+
+					// If seed is empty or no tokens remain, we can return early
+					tokens.splice( i, 1 );
+					selector = seed.length && toSelector( tokens );
+					if ( !selector ) {
+						push.apply( results, seed );
+						return results;
+					}
+
+					break;
+				}
+			}
+		}
+	}
+
+	// Compile and execute a filtering function if one is not provided
+	// Provide `match` to avoid retokenization if we modified the selector above
+	( compiled || compile( selector, match ) )(
+		seed,
+		context,
+		!documentIsHTML,
+		results,
+		rsibling.test( selector ) && testContext( context.parentNode ) || context
+	);
+	return results;
+};
+
+// One-time assignments
+
+// Sort stability
+support.sortStable = expando.split("").sort( sortOrder ).join("") === expando;
+
+// Support: Chrome 14-35+
+// Always assume duplicates if they aren't passed to the comparison function
+support.detectDuplicates = !!hasDuplicate;
+
+// Initialize against the default document
+setDocument();
+
+// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27)
+// Detached nodes confoundingly follow *each other*
+support.sortDetached = assert(function( div1 ) {
+	// Should return 1, but returns 4 (following)
+	return div1.compareDocumentPosition( document.createElement("div") ) & 1;
+});
+
+// Support: IE<8
+// Prevent attribute/property "interpolation"
+// http://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
+if ( !assert(function( div ) {
+	div.innerHTML = "<a href='#'></a>";
+	return div.firstChild.getAttribute("href") === "#" ;
+}) ) {
+	addHandle( "type|href|height|width", function( elem, name, isXML ) {
+		if ( !isXML ) {
+			return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
+		}
+	});
+}
+
+// Support: IE<9
+// Use defaultValue in place of getAttribute("value")
+if ( !support.attributes || !assert(function( div ) {
+	div.innerHTML = "<input/>";
+	div.firstChild.setAttribute( "value", "" );
+	return div.firstChild.getAttribute( "value" ) === "";
+}) ) {
+	addHandle( "value", function( elem, name, isXML ) {
+		if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
+			return elem.defaultValue;
+		}
+	});
+}
+
+// Support: IE<9
+// Use getAttributeNode to fetch booleans when getAttribute lies
+if ( !assert(function( div ) {
+	return div.getAttribute("disabled") == null;
+}) ) {
+	addHandle( booleans, function( elem, name, isXML ) {
+		var val;
+		if ( !isXML ) {
+			return elem[ name ] === true ? name.toLowerCase() :
+					(val = elem.getAttributeNode( name )) && val.specified ?
+					val.value :
+				null;
+		}
+	});
+}
+
+return Sizzle;
+
+})( window );
+
+
+
+jQuery.find = Sizzle;
+jQuery.expr = Sizzle.selectors;
+jQuery.expr[":"] = jQuery.expr.pseudos;
+jQuery.unique = Sizzle.uniqueSort;
+jQuery.text = Sizzle.getText;
+jQuery.isXMLDoc = Sizzle.isXML;
+jQuery.contains = Sizzle.contains;
+
+
+
+var rneedsContext = jQuery.expr.match.needsContext;
+
+var rsingleTag = (/^<(\w+)\s*\/?>(?:<\/\1>|)$/);
+
+
+
+var risSimple = /^.[^:#\[\.,]*$/;
+
+// Implement the identical functionality for filter and not
+function winnow( elements, qualifier, not ) {
+	if ( jQuery.isFunction( qualifier ) ) {
+		return jQuery.grep( elements, function( elem, i ) {
+			/* jshint -W018 */
+			return !!qualifier.call( elem, i, elem ) !== not;
+		});
+
+	}
+
+	if ( qualifier.nodeType ) {
+		return jQuery.grep( elements, function( elem ) {
+			return ( elem === qualifier ) !== not;
+		});
+
+	}
+
+	if ( typeof qualifier === "string" ) {
+		if ( risSimple.test( qualifier ) ) {
+			return jQuery.filter( qualifier, elements, not );
+		}
+
+		qualifier = jQuery.filter( qualifier, elements );
+	}
+
+	return jQuery.grep( elements, function( elem ) {
+		return ( indexOf.call( qualifier, elem ) >= 0 ) !== not;
+	});
+}
+
+jQuery.filter = function( expr, elems, not ) {
+	var elem = elems[ 0 ];
+
+	if ( not ) {
+		expr = ":not(" + expr + ")";
+	}
+
+	return elems.length === 1 && elem.nodeType === 1 ?
+		jQuery.find.matchesSelector( elem, expr ) ? [ elem ] : [] :
+		jQuery.find.matches( expr, jQuery.grep( elems, function( elem ) {
+			return elem.nodeType === 1;
+		}));
+};
+
+jQuery.fn.extend({
+	find: function( selector ) {
+		var i,
+			len = this.length,
+			ret = [],
+			self = this;
+
+		if ( typeof selector !== "string" ) {
+			return this.pushStack( jQuery( selector ).filter(function() {
+				for ( i = 0; i < len; i++ ) {
+					if ( jQuery.contains( self[ i ], this ) ) {
+						return true;
+					}
+				}
+			}) );
+		}
+
+		for ( i = 0; i < len; i++ ) {
+			jQuery.find( selector, self[ i ], ret );
+		}
+
+		// Needed because $( selector, context ) becomes $( context ).find( selector )
+		ret = this.pushStack( len > 1 ? jQuery.unique( ret ) : ret );
+		ret.selector = this.selector ? this.selector + " " + selector : selector;
+		return ret;
+	},
+	filter: function( selector ) {
+		return this.pushStack( winnow(this, selector || [], false) );
+	},
+	not: function( selector ) {
+		return this.pushStack( winnow(this, selector || [], true) );
+	},
+	is: function( selector ) {
+		return !!winnow(
+			this,
+
+			// If this is a positional/relative selector, check membership in the returned set
+			// so $("p:first").is("p:last") won't return true for a doc with two "p".
+			typeof selector === "string" && rneedsContext.test( selector ) ?
+				jQuery( selector ) :
+				selector || [],
+			false
+		).length;
+	}
+});
+
+
+// Initialize a jQuery object
+
+
+// A central reference to the root jQuery(document)
+var rootjQuery,
+
+	// A simple way to check for HTML strings
+	// Prioritize #id over <tag> to avoid XSS via location.hash (#9521)
+	// Strict HTML recognition (#11290: must start with <)
+	rquickExpr = /^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,
+
+	init = jQuery.fn.init = function( selector, context ) {
+		var match, elem;
+
+		// HANDLE: $(""), $(null), $(undefined), $(false)
+		if ( !selector ) {
+			return this;
+		}
+
+		// Handle HTML strings
+		if ( typeof selector === "string" ) {
+			if ( selector[0] === "<" && selector[ selector.length - 1 ] === ">" && selector.length >= 3 ) {
+				// Assume that strings that start and end with <> are HTML and skip the regex check
+				match = [ null, selector, null ];
+
+			} else {
+				match = rquickExpr.exec( selector );
+			}
+
+			// Match html or make sure no context is specified for #id
+			if ( match && (match[1] || !context) ) {
+
+				// HANDLE: $(html) -> $(array)
+				if ( match[1] ) {
+					context = context instanceof jQuery ? context[0] : context;
+
+					// Option to run scripts is true for back-compat
+					// Intentionally let the error be thrown if parseHTML is not present
+					jQuery.merge( this, jQuery.parseHTML(
+						match[1],
+						context && context.nodeType ? context.ownerDocument || context : document,
+						true
+					) );
+
+					// HANDLE: $(html, props)
+					if ( rsingleTag.test( match[1] ) && jQuery.isPlainObject( context ) ) {
+						for ( match in context ) {
+							// Properties of context are called as methods if possible
+							if ( jQuery.isFunction( this[ match ] ) ) {
+								this[ match ]( context[ match ] );
+
+							// ...and otherwise set as attributes
+							} else {
+								this.attr( match, context[ match ] );
+							}
+						}
+					}
+
+					return this;
+
+				// HANDLE: $(#id)
+				} else {
+					elem = document.getElementById( match[2] );
+
+					// Support: Blackberry 4.6
+					// gEBID returns nodes no longer in the document (#6963)
+					if ( elem && elem.parentNode ) {
+						// Inject the element directly into the jQuery object
+						this.length = 1;
+						this[0] = elem;
+					}
+
+					this.context = document;
+					this.selector = selector;
+					return this;
+				}
+
+			// HANDLE: $(expr, $(...))
+			} else if ( !context || context.jquery ) {
+				return ( context || rootjQuery ).find( selector );
+
+			// HANDLE: $(expr, context)
+			// (which is just equivalent to: $(context).find(expr)
+			} else {
+				return this.constructor( context ).find( selector );
+			}
+
+		// HANDLE: $(DOMElement)
+		} else if ( selector.nodeType ) {
+			this.context = this[0] = selector;
+			this.length = 1;
+			return this;
+
+		// HANDLE: $(function)
+		// Shortcut for document ready
+		} else if ( jQuery.isFunction( selector ) ) {
+			return typeof rootjQuery.ready !== "undefined" ?
+				rootjQuery.ready( selector ) :
+				// Execute immediately if ready is not present
+				selector( jQuery );
+		}
+
+		if ( selector.selector !== undefined ) {
+			this.selector = selector.selector;
+			this.context = selector.context;
+		}
+
+		return jQuery.makeArray( selector, this );
+	};
+
+// Give the init function the jQuery prototype for later instantiation
+init.prototype = jQuery.fn;
+
+// Initialize central reference
+rootjQuery = jQuery( document );
+
+
+var rparentsprev = /^(?:parents|prev(?:Until|All))/,
+	// Methods guaranteed to produce a unique set when starting from a unique set
+	guaranteedUnique = {
+		children: true,
+		contents: true,
+		next: true,
+		prev: true
+	};
+
+jQuery.extend({
+	dir: function( elem, dir, until ) {
+		var matched = [],
+			truncate = until !== undefined;
+
+		while ( (elem = elem[ dir ]) && elem.nodeType !== 9 ) {
+			if ( elem.nodeType === 1 ) {
+				if ( truncate && jQuery( elem ).is( until ) ) {
+					break;
+				}
+				matched.push( elem );
+			}
+		}
+		return matched;
+	},
+
+	sibling: function( n, elem ) {
+		var matched = [];
+
+		for ( ; n; n = n.nextSibling ) {
+			if ( n.nodeType === 1 && n !== elem ) {
+				matched.push( n );
+			}
+		}
+
+		return matched;
+	}
+});
+
+jQuery.fn.extend({
+	has: function( target ) {
+		var targets = jQuery( target, this ),
+			l = targets.length;
+
+		return this.filter(function() {
+			var i = 0;
+			for ( ; i < l; i++ ) {
+				if ( jQuery.contains( this, targets[i] ) ) {
+					return true;
+				}
+			}
+		});
+	},
+
+	closest: function( selectors, context ) {
+		var cur,
+			i = 0,
+			l = this.length,
+			matched = [],
+			pos = rneedsContext.test( selectors ) || typeof selectors !== "string" ?
+				jQuery( selectors, context || this.context ) :
+				0;
+
+		for ( ; i < l; i++ ) {
+			for ( cur = this[i]; cur && cur !== context; cur = cur.parentNode ) {
+				// Always skip document fragments
+				if ( cur.nodeType < 11 && (pos ?
+					pos.index(cur) > -1 :
+
+					// Don't pass non-elements to Sizzle
+					cur.nodeType === 1 &&
+						jQuery.find.matchesSelector(cur, selectors)) ) {
+
+					matched.push( cur );
+					break;
+				}
+			}
+		}
+
+		return this.pushStack( matched.length > 1 ? jQuery.unique( matched ) : matched );
+	},
+
+	// Determine the position of an element within the set
+	index: function( elem ) {
+
+		// No argument, return index in parent
+		if ( !elem ) {
+			return ( this[ 0 ] && this[ 0 ].parentNode ) ? this.first().prevAll().length : -1;
+		}
+
+		// Index in selector
+		if ( typeof elem === "string" ) {
+			return indexOf.call( jQuery( elem ), this[ 0 ] );
+		}
+
+		// Locate the position of the desired element
+		return indexOf.call( this,
+
+			// If it receives a jQuery object, the first element is used
+			elem.jquery ? elem[ 0 ] : elem
+		);
+	},
+
+	add: function( selector, context ) {
+		return this.pushStack(
+			jQuery.unique(
+				jQuery.merge( this.get(), jQuery( selector, context ) )
+			)
+		);
+	},
+
+	addBack: function( selector ) {
+		return this.add( selector == null ?
+			this.prevObject : this.prevObject.filter(selector)
+		);
+	}
+});
+
+function sibling( cur, dir ) {
+	while ( (cur = cur[dir]) && cur.nodeType !== 1 ) {}
+	return cur;
+}
+
+jQuery.each({
+	parent: function( elem ) {
+		var parent = elem.parentNode;
+		return parent && parent.nodeType !== 11 ? parent : null;
+	},
+	parents: function( elem ) {
+		return jQuery.dir( elem, "parentNode" );
+	},
+	parentsUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "parentNode", until );
+	},
+	next: function( elem ) {
+		return sibling( elem, "nextSibling" );
+	},
+	prev: function( elem ) {
+		return sibling( elem, "previousSibling" );
+	},
+	nextAll: function( elem ) {
+		return jQuery.dir( elem, "nextSibling" );
+	},
+	prevAll: function( elem ) {
+		return jQuery.dir( elem, "previousSibling" );
+	},
+	nextUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "nextSibling", until );
+	},
+	prevUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "previousSibling", until );
+	},
+	siblings: function( elem ) {
+		return jQuery.sibling( ( elem.parentNode || {} ).firstChild, elem );
+	},
+	children: function( elem ) {
+		return jQuery.sibling( elem.firstChild );
+	},
+	contents: function( elem ) {
+		return elem.contentDocument || jQuery.merge( [], elem.childNodes );
+	}
+}, function( name, fn ) {
+	jQuery.fn[ name ] = function( until, selector ) {
+		var matched = jQuery.map( this, fn, until );
+
+		if ( name.slice( -5 ) !== "Until" ) {
+			selector = until;
+		}
+
+		if ( selector && typeof selector === "string" ) {
+			matched = jQuery.filter( selector, matched );
+		}
+
+		if ( this.length > 1 ) {
+			// Remove duplicates
+			if ( !guaranteedUnique[ name ] ) {
+				jQuery.unique( matched );
+			}
+
+			// Reverse order for parents* and prev-derivatives
+			if ( rparentsprev.test( name ) ) {
+				matched.reverse();
+			}
+		}
+
+		return this.pushStack( matched );
+	};
+});
+var rnotwhite = (/\S+/g);
+
+
+
+// String to Object options format cache
+var optionsCache = {};
+
+// Convert String-formatted options into Object-formatted ones and store in cache
+function createOptions( options ) {
+	var object = optionsCache[ options ] = {};
+	jQuery.each( options.match( rnotwhite ) || [], function( _, flag ) {
+		object[ flag ] = true;
+	});
+	return object;
+}
+
+/*
+ * Create a callback list using the following parameters:
+ *
+ *	options: an optional list of space-separated options that will change how
+ *			the callback list behaves or a more traditional option object
+ *
+ * By default a callback list will act like an event callback list and can be
+ * "fired" multiple times.
+ *
+ * Possible options:
+ *
+ *	once:			will ensure the callback list can only be fired once (like a Deferred)
+ *
+ *	memory:			will keep track of previous values and will call any callback added
+ *					after the list has been fired right away with the latest "memorized"
+ *					values (like a Deferred)
+ *
+ *	unique:			will ensure a callback can only be added once (no duplicate in the list)
+ *
+ *	stopOnFalse:	interrupt callings when a callback returns false
+ *
+ */
+jQuery.Callbacks = function( options ) {
+
+	// Convert options from String-formatted to Object-formatted if needed
+	// (we check in cache first)
+	options = typeof options === "string" ?
+		( optionsCache[ options ] || createOptions( options ) ) :
+		jQuery.extend( {}, options );
+
+	var // Last fire value (for non-forgettable lists)
+		memory,
+		// Flag to know if list was already fired
+		fired,
+		// Flag to know if list is currently firing
+		firing,
+		// First callback to fire (used internally by add and fireWith)
+		firingStart,
+		// End of the loop when firing
+		firingLength,
+		// Index of currently firing callback (modified by remove if needed)
+		firingIndex,
+		// Actual callback list
+		list = [],
+		// Stack of fire calls for repeatable lists
+		stack = !options.once && [],
+		// Fire callbacks
+		fire = function( data ) {
+			memory = options.memory && data;
+			fired = true;
+			firingIndex = firingStart || 0;
+			firingStart = 0;
+			firingLength = list.length;
+			firing = true;
+			for ( ; list && firingIndex < firingLength; firingIndex++ ) {
+				if ( list[ firingIndex ].apply( data[ 0 ], data[ 1 ] ) === false && options.stopOnFalse ) {
+					memory = false; // To prevent further calls using add
+					break;
+				}
+			}
+			firing = false;
+			if ( list ) {
+				if ( stack ) {
+					if ( stack.length ) {
+						fire( stack.shift() );
+					}
+				} else if ( memory ) {
+					list = [];
+				} else {
+					self.disable();
+				}
+			}
+		},
+		// Actual Callbacks object
+		self = {
+			// Add a callback or a collection of callbacks to the list
+			add: function() {
+				if ( list ) {
+					// First, we save the current length
+					var start = list.length;
+					(function add( args ) {
+						jQuery.each( args, function( _, arg ) {
+							var type = jQuery.type( arg );
+							if ( type === "function" ) {
+								if ( !options.unique || !self.has( arg ) ) {
+									list.push( arg );
+								}
+							} else if ( arg && arg.length && type !== "string" ) {
+								// Inspect recursively
+								add( arg );
+							}
+						});
+					})( arguments );
+					// Do we need to add the callbacks to the
+					// current firing batch?
+					if ( firing ) {
+						firingLength = list.l

<TRUNCATED>

[30/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
deleted file mode 100644
index 79512d7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.streaming.api.state.StreamStateHandle;
-
-import java.io.InputStream;
-
-/**
- * A state handle that points to state in a file system, accessible as an input stream.
- */
-public class FileStreamStateHandle extends AbstractFileState implements StreamStateHandle {
-	
-	private static final long serialVersionUID = -6826990484549987311L;
-
-	/**
-	 * Creates a new FileStreamStateHandle pointing to state at the given file path.
-	 * 
-	 * @param filePath The path to the file containing the checkpointed state.
-	 */
-	public FileStreamStateHandle(Path filePath) {
-		super(filePath);
-	}
-
-	@Override
-	public InputStream getState(ClassLoader userCodeClassLoader) throws Exception {
-		return getFileSystem().open(getFilePath());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
deleted file mode 100644
index 107a3be..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.state.AbstractHeapKvState;
-
-import java.io.DataOutputStream;
-import java.util.HashMap;
-
-/**
- * Heap-backed key/value state that is snapshotted into files.
- * 
- * @param <K> The type of the key.
- * @param <V> The type of the value.
- */
-public class FsHeapKvState<K, V> extends AbstractHeapKvState<K, V, FsStateBackend> {
-	
-	/** The file system state backend backing snapshots of this state */
-	private final FsStateBackend backend;
-	
-	/**
-	 * Creates a new and empty key/value state.
-	 * 
-	 * @param keySerializer The serializer for the key.
-	 * @param valueSerializer The serializer for the value.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
-	 * @param backend The file system state backend backing snapshots of this state
-	 */
-	public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
-							V defaultValue, FsStateBackend backend) {
-		super(keySerializer, valueSerializer, defaultValue);
-		this.backend = backend;
-	}
-
-	/**
-	 * Creates a new key/value state with the given state contents.
-	 * This method is used to re-create key/value state with existing data, for example from
-	 * a snapshot.
-	 * 
-	 * @param keySerializer The serializer for the key.
-	 * @param valueSerializer The serializer for the value.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
-	 * @param state The map of key/value pairs to initialize the state with.
-	 * @param backend The file system state backend backing snapshots of this state
-	 */
-	public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
-							V defaultValue, HashMap<K, V> state, FsStateBackend backend) {
-		super(keySerializer, valueSerializer, defaultValue, state);
-		this.backend = backend;
-	}
-
-	
-	@Override
-	public FsHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
-		// first, create an output stream to write to
-		try (FsStateBackend.FsCheckpointStateOutputStream out = 
-					backend.createCheckpointStateOutputStream(checkpointId, timestamp)) {
-
-			// serialize the state to the output stream
-			OutputViewDataOutputStreamWrapper outView = 
-					new OutputViewDataOutputStreamWrapper(new DataOutputStream(out));
-			outView.writeInt(size());
-			writeStateToOutputView(outView);
-			outView.flush();
-			
-			// create a handle to the state
-			return new FsHeapKvStateSnapshot<>(getKeySerializer(), getValueSerializer(), out.closeAndGetPath());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
deleted file mode 100644
index c7117f8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.streaming.api.state.KvStateSnapshot;
-
-import java.io.DataInputStream;
-import java.util.HashMap;
-
-/**
- * A snapshot of a heap key/value state stored in a file.
- * 
- * @param <K> The type of the key in the snapshot state.
- * @param <V> The type of the value in the snapshot state.
- */
-public class FsHeapKvStateSnapshot<K, V> extends AbstractFileState implements KvStateSnapshot<K, V, FsStateBackend> {
-	
-	private static final long serialVersionUID = 1L;
-
-	/** Name of the key serializer class */
-	private final String keySerializerClassName;
-
-	/** Name of the value serializer class */
-	private final String valueSerializerClassName;
-
-	/**
-	 * Creates a new state snapshot with data in the file system.
-	 *
-	 * @param keySerializer The serializer for the keys.
-	 * @param valueSerializer The serializer for the values.
-	 * @param filePath The path where the snapshot data is stored.
-	 */
-	public FsHeapKvStateSnapshot(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, Path filePath) {
-		super(filePath);
-		this.keySerializerClassName = keySerializer.getClass().getName();
-		this.valueSerializerClassName = valueSerializer.getClass().getName();
-	}
-
-	@Override
-	public FsHeapKvState<K, V> restoreState(
-			FsStateBackend stateBackend,
-			final TypeSerializer<K> keySerializer,
-			final TypeSerializer<V> valueSerializer,
-			V defaultValue,
-			ClassLoader classLoader) throws Exception {
-
-		// validity checks
-		if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
-				!valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
-			throw new IllegalArgumentException(
-					"Cannot restore the state from the snapshot with the given serializers. " +
-							"State (K/V) was serialized with (" + valueSerializerClassName +
-							"/" + keySerializerClassName + ")");
-		}
-		
-		// state restore
-		try (FSDataInputStream inStream = stateBackend.getFileSystem().open(getFilePath())) {
-			InputViewDataInputStreamWrapper inView = new InputViewDataInputStreamWrapper(new DataInputStream(inStream));
-			
-			final int numEntries = inView.readInt();
-			HashMap<K, V> stateMap = new HashMap<>(numEntries);
-			
-			for (int i = 0; i < numEntries; i++) {
-				K key = keySerializer.deserialize(inView);
-				V value = valueSerializer.deserialize(inView);
-				stateMap.put(key, value);
-			}
-			
-			return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap, stateBackend);
-		}
-		catch (Exception e) {
-			throw new Exception("Failed to restore state from file system", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
deleted file mode 100644
index 3cbd227..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
+++ /dev/null
@@ -1,412 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.StateBackend;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.UUID;
-
-/**
- * The file state backend is a state backend that stores the state of streaming jobs in a file system.
- * 
- * <p>The state backend has one core directory into which it puts all checkpoint data. Inside that
- * directory, it creates a directory per job, inside which each checkpoint gets a directory, with
- * files for each state, for example:
- * 
- * {@code hdfs://namenode:port/flink-checkpoints/<job-id>/chk-17/6ba7b810-9dad-11d1-80b4-00c04fd430c8 }
- */
-public class FsStateBackend extends StateBackend<FsStateBackend> {
-
-	private static final long serialVersionUID = -8191916350224044011L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FsStateBackend.class);
-	
-	
-	/** The path to the directory for the checkpoint data, including the file system
-	 * description via scheme and optional authority */
-	private final Path basePath;
-	
-	/** The directory (job specific) into this initialized instance of the backend stores its data */
-	private transient Path checkpointDirectory;
-	
-	/** Cached handle to the file system for file operations */
-	private transient FileSystem filesystem;
-
-
-	/**
-	 * Creates a new state backend that stores its checkpoint data in the file system and location
-	 * defined by the given URI.
-	 *
-	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
-	 * must be accessible via {@link FileSystem#get(URI)}.
-	 *
-	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
-	 * (host and port), or that the Hadoop configuration that describes that information must be in the
-	 * classpath.
-	 *
-	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
-	 *                          and the path to teh checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
-	 */
-	public FsStateBackend(String checkpointDataUri) throws IOException {
-		this(new Path(checkpointDataUri));
-	}
-
-	/**
-	 * Creates a new state backend that stores its checkpoint data in the file system and location
-	 * defined by the given URI.
-	 *
-	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
-	 * must be accessible via {@link FileSystem#get(URI)}.
-	 *
-	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
-	 * (host and port), or that the Hadoop configuration that describes that information must be in the
-	 * classpath.
-	 *
-	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
-	 *                          and the path to teh checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
-	 */
-	public FsStateBackend(Path checkpointDataUri) throws IOException {
-		this(checkpointDataUri.toUri());
-	}
-
-	/**
-	 * Creates a new state backend that stores its checkpoint data in the file system and location
-	 * defined by the given URI.
-	 * 
-	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
-	 * must be accessible via {@link FileSystem#get(URI)}.
-	 * 
-	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
-	 * (host and port), or that the Hadoop configuration that describes that information must be in the
-	 * classpath.
-	 * 
-	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
-	 *                          and the path to teh checkpoint data directory.
-	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
-	 */
-	public FsStateBackend(URI checkpointDataUri) throws IOException {
-		final String scheme = checkpointDataUri.getScheme();
-		final String path = checkpointDataUri.getPath();
-		
-		// some validity checks
-		if (scheme == null) {
-			throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " +
-					"Please specify the file system scheme explicitly in the URI.");
-		}
-		if (path == null) {
-			throw new IllegalArgumentException("The path to store the checkpoint data in is null. " +
-					"Please specify a directory path for the checkpoint data.");
-		}
-		if (path.length() == 0 || path.equals("/")) {
-			throw new IllegalArgumentException("Cannot use the root directory for checkpoints.");
-		}
-		
-		// we do a bit of work to make sure that the URI for the filesystem refers to exactly the same
-		// (distributed) filesystem on all hosts and includes full host/port information, even if the
-		// original URI did not include that. We count on the filesystem loading from the configuration
-		// to fill in the missing data.
-		
-		// try to grab the file system for this path/URI
-		this.filesystem = FileSystem.get(checkpointDataUri);
-		if (this.filesystem == null) {
-			throw new IOException("Could not find a file system for the given scheme in the available configurations.");
-		}
-
-		URI fsURI = this.filesystem.getUri();
-		try {
-			URI baseURI = new URI(fsURI.getScheme(), fsURI.getAuthority(), path, null, null);
-			this.basePath = new Path(baseURI);
-		}
-		catch (URISyntaxException e) {
-			throw new IOException(
-					String.format("Cannot create file system URI for checkpointDataUri %s and filesystem URI %s", 
-							checkpointDataUri, fsURI), e);
-		}
-	}
-
-	/**
-	 * Gets the base directory where all state-containing files are stored.
-	 * The job specific directory is created inside this directory.
-	 * 
-	 * @return The base directory.
-	 */
-	public Path getBasePath() {
-		return basePath;
-	}
-
-	/**
-	 * Gets the directory where this state backend stores its checkpoint data. Will be null if
-	 * the state backend has not been initialized.
-	 * 
-	 * @return The directory where this state backend stores its checkpoint data.
-	 */
-	public Path getCheckpointDirectory() {
-		return checkpointDirectory;
-	}
-
-	/**
-	 * Checks whether this state backend is initialized. Note that initialization does not carry
-	 * across serialization. After each serialization, the state backend needs to be initialized.
-	 * 
-	 * @return True, if the file state backend has been initialized, false otherwise.
-	 */
-	public boolean isInitialized() {
-		return filesystem != null && checkpointDirectory != null; 
-	}
-
-	/**
-	 * Gets the file system handle for the file system that stores the state for this backend.
-	 * 
-	 * @return This backend's file system handle.
-	 */
-	public FileSystem getFileSystem() {
-		if (filesystem != null) {
-			return filesystem;
-		}
-		else {
-			throw new IllegalStateException("State backend has not been initialized.");
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  initialization and cleanup
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public void initializeForJob(JobID jobId) throws Exception {
-		Path dir = new Path(basePath, jobId.toString());
-		
-		LOG.info("Initializing file state backend to URI " + dir);
-		
-		filesystem = basePath.getFileSystem();
-		filesystem.mkdirs(dir);
-
-		checkpointDirectory = dir;
-	}
-
-	@Override
-	public void disposeAllStateForCurrentJob() throws Exception {
-		FileSystem fs = this.filesystem;
-		Path dir = this.checkpointDirectory;
-		
-		if (fs != null && dir != null) {
-			this.filesystem = null;
-			this.checkpointDirectory = null;
-			fs.delete(dir, true);
-		}
-		else {
-			throw new IllegalStateException("state backend has not been initialized");
-		}
-	}
-
-	@Override
-	public void close() throws Exception {}
-
-	// ------------------------------------------------------------------------
-	//  state backend operations
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public <K, V> FsHeapKvState<K, V> createKvState(
-			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) throws Exception {
-		return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, this);
-	}
-
-	@Override
-	public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
-			S state, long checkpointID, long timestamp) throws Exception
-	{
-		checkFileSystemInitialized();
-
-		// make sure the directory for that specific checkpoint exists
-		final Path checkpointDir = createCheckpointDirPath(checkpointID);
-		filesystem.mkdirs(checkpointDir);
-
-		
-		Exception latestException = null;
-
-		for (int attempt = 0; attempt < 10; attempt++) {
-			Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
-			FSDataOutputStream outStream;
-			try {
-				outStream = filesystem.create(targetPath, false);
-			}
-			catch (Exception e) {
-				latestException = e;
-				continue;
-			}
-
-			ObjectOutputStream os = new ObjectOutputStream(outStream);
-			os.writeObject(state);
-			os.close();
-			return new FileSerializableStateHandle<S>(targetPath);
-		}
-		
-		throw new Exception("Could not open output stream for state backend", latestException);
-	}
-	
-	@Override
-	public FsCheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp) throws Exception {
-		checkFileSystemInitialized();
-		
-		final Path checkpointDir = createCheckpointDirPath(checkpointID);
-		filesystem.mkdirs(checkpointDir);
-		
-		Exception latestException = null;
-		
-		for (int attempt = 0; attempt < 10; attempt++) {
-			Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
-			try {
-				FSDataOutputStream outStream = filesystem.create(targetPath, false);
-				return new FsCheckpointStateOutputStream(outStream, targetPath, filesystem);
-			}
-			catch (Exception e) {
-				latestException = e;
-			}
-		}
-		throw new Exception("Could not open output stream for state backend", latestException);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  utilities
-	// ------------------------------------------------------------------------
-
-	private void checkFileSystemInitialized() throws IllegalStateException {
-		if (filesystem == null || checkpointDirectory == null) {
-			throw new IllegalStateException("filesystem has not been re-initialized after deserialization");
-		}
-	}
-	
-	private Path createCheckpointDirPath(long checkpointID) {
-		return new Path(checkpointDirectory, "chk-" + checkpointID);
-	}
-	
-	@Override
-	public String toString() {
-		return checkpointDirectory == null ?
-			"File State Backend @ " + basePath : 
-			"File State Backend (initialized) @ " + checkpointDirectory;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Output stream for state checkpointing
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A CheckpointStateOutputStream that writes into a file and returns the path to that file upon
-	 * closing.
-	 */
-	public static final class FsCheckpointStateOutputStream extends CheckpointStateOutputStream {
-
-		private final FSDataOutputStream outStream;
-		
-		private final Path filePath;
-		
-		private final FileSystem fs;
-		
-		private boolean closed;
-
-		FsCheckpointStateOutputStream(FSDataOutputStream outStream, Path filePath, FileSystem fs) {
-			this.outStream = outStream;
-			this.filePath = filePath;
-			this.fs = fs;
-		}
-
-
-		@Override
-		public void write(int b) throws IOException {
-			outStream.write(b);
-		}
-
-		@Override
-		public void write(byte[] b, int off, int len) throws IOException {
-			outStream.write(b, off, len);
-		}
-
-		@Override
-		public void flush() throws IOException {
-			outStream.flush();
-		}
-
-		/**
-		 * If the stream is only closed, we remove the produced file (cleanup through the auto close
-		 * feature, for example). This method throws no exception if the deletion fails, but only
-		 * logs the error.
-		 */
-		@Override
-		public void close() {
-			synchronized (this) {
-				if (!closed) {
-					closed = true;
-					try {
-						outStream.close();
-						fs.delete(filePath, false);
-						
-						// attempt to delete the parent (will fail and be ignored if the parent has more files)
-						try {
-							fs.delete(filePath.getParent(), false);
-						} catch (IOException ignored) {}
-					}
-					catch (Exception e) {
-						LOG.warn("Cannot delete closed and discarded state stream to " + filePath, e);
-					}
-				}
-			}
-		}
-
-		@Override
-		public FileStreamStateHandle closeAndGetHandle() throws IOException {
-			return new FileStreamStateHandle(closeAndGetPath());
-		}
-
-		/**
-		 * Closes the stream and returns the path to the file that contains the stream's data.
-		 * @return The path to the file that contains the stream's data.
-		 * @throws IOException Thrown if the stream cannot be successfully closed.
-		 */
-		public Path closeAndGetPath() throws IOException {
-			synchronized (this) {
-				if (!closed) {
-					closed = true;
-					outStream.close();
-					return filePath;
-				}
-				else {
-					throw new IOException("Stream has already been closed and discarded.");
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
deleted file mode 100644
index f0ad6bd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.streaming.api.state.StateBackendFactory;
-
-/**
- * A factory that creates an {@link org.apache.flink.streaming.api.state.filesystem.FsStateBackend}
- * from a configuration.
- */
-public class FsStateBackendFactory implements StateBackendFactory<FsStateBackend> {
-	
-	/** The key under which the config stores the directory where checkpoints should be stored */
-	public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
-	
-	
-	@Override
-	public FsStateBackend createFromConfig(Configuration config) throws Exception {
-		String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
-
-		if (checkpointDirURI == null) {
-			throw new IllegalConfigurationException(
-					"Cannot create the file system state backend: The configuration does not specify the " +
-							"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
-		}
-		
-		try {
-			Path path = new Path(checkpointDirURI);
-			return new FsStateBackend(path);
-		}
-		catch (IllegalArgumentException e) {
-			throw new Exception("Cannot initialize File System State Backend with URI '"
-					+ checkpointDirURI + '.', e);
-		}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
deleted file mode 100644
index 7952e58..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.memory;
-
-import org.apache.flink.streaming.api.state.StreamStateHandle;
-
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-
-/**
- * A state handle that contains stream state in a byte array.
- */
-public final class ByteStreamStateHandle implements StreamStateHandle {
-
-	private static final long serialVersionUID = -5280226231200217594L;
-	
-	/** the state data */
-	private final byte[] data;
-
-	/**
-	 * Creates a new ByteStreamStateHandle containing the given data.
-	 * 
-	 * @param data The state data.
-	 */
-	public ByteStreamStateHandle(byte[] data) {
-		this.data = data;
-	}
-
-	@Override
-	public InputStream getState(ClassLoader userCodeClassLoader) {
-		return new ByteArrayInputStream(data);
-	}
-
-	@Override
-	public void discardState() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
deleted file mode 100644
index e611887..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.memory;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-import org.apache.flink.streaming.api.state.AbstractHeapKvState;
-
-import java.util.HashMap;
-
-/**
- * Heap-backed key/value state that is snapshotted into a serialized memory copy.
- *
- * @param <K> The type of the key.
- * @param <V> The type of the value.
- */
-public class MemHeapKvState<K, V> extends AbstractHeapKvState<K, V, MemoryStateBackend> {
-	
-	public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
-		super(keySerializer, valueSerializer, defaultValue);
-	}
-
-	public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
-							V defaultValue, HashMap<K, V> state) {
-		super(keySerializer, valueSerializer, defaultValue, state);
-	}
-	
-	@Override
-	public MemoryHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
-		DataOutputSerializer ser = new DataOutputSerializer(Math.max(size() * 16, 16));
-		writeStateToOutputView(ser);
-		byte[] bytes = ser.getCopyOfBuffer();
-		
-		return new MemoryHeapKvStateSnapshot<K, V>(getKeySerializer(), getValueSerializer(), bytes, size());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
deleted file mode 100644
index 7f50379..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.memory;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.streaming.api.state.KvStateSnapshot;
-
-import java.util.HashMap;
-
-/**
- * A snapshot of a {@link MemHeapKvState} for a checkpoint. The data is stored in a heap byte
- * array, in serialized form.
- * 
- * @param <K> The type of the key in the snapshot state.
- * @param <V> The type of the value in the snapshot state.
- */
-public class MemoryHeapKvStateSnapshot<K, V> implements KvStateSnapshot<K, V, MemoryStateBackend> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	/** Name of the key serializer class */
-	private final String keySerializerClassName;
-
-	/** Name of the value serializer class */
-	private final String valueSerializerClassName;
-	
-	/** The serialized data of the state key/value pairs */
-	private final byte[] data;
-	
-	/** The number of key/value pairs */
-	private final int numEntries;
-
-	/**
-	 * Creates a new heap memory state snapshot.
-	 *
-	 * @param keySerializer The serializer for the keys.
-	 * @param valueSerializer The serializer for the values.
-	 * @param data The serialized data of the state key/value pairs
-	 * @param numEntries The number of key/value pairs
-	 */
-	public MemoryHeapKvStateSnapshot(TypeSerializer<K> keySerializer,
-						TypeSerializer<V> valueSerializer, byte[] data, int numEntries) {
-		this.keySerializerClassName = keySerializer.getClass().getName();
-		this.valueSerializerClassName = valueSerializer.getClass().getName();
-		this.data = data;
-		this.numEntries = numEntries;
-	}
-
-
-	@Override
-	public MemHeapKvState<K, V> restoreState(
-			MemoryStateBackend stateBackend,
-			final TypeSerializer<K> keySerializer,
-			final TypeSerializer<V> valueSerializer,
-			V defaultValue,
-			ClassLoader classLoader) throws Exception {
-
-		// validity checks
-		if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
-			!valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
-				throw new IllegalArgumentException(
-						"Cannot restore the state from the snapshot with the given serializers. " +
-						"State (K/V) was serialized with (" + valueSerializerClassName + 
-						"/" + keySerializerClassName + ")");
-		}
-		
-		// restore state
-		HashMap<K, V> stateMap = new HashMap<>(numEntries);
-		DataInputDeserializer in = new DataInputDeserializer(data, 0, data.length);
-		
-		for (int i = 0; i < numEntries; i++) {
-			K key = keySerializer.deserialize(in);
-			V value = valueSerializer.deserialize(in);
-			stateMap.put(key, value);
-		}
-		
-		return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap);
-	}
-
-	/**
-	 * Discarding the heap state is a no-op.
-	 */
-	@Override
-	public void discardState() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
deleted file mode 100644
index 05368bd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.memory;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.StreamStateHandle;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * A {@link StateBackend} that stores all its data and checkpoints in memory and has no
- * capabilities to spill to disk. Checkpoints are serialized and the serialized data is
- * transferred 
- */
-public class MemoryStateBackend extends StateBackend<MemoryStateBackend> {
-
-	private static final long serialVersionUID = 4109305377809414635L;
-	
-	/** The default maximal size that the snapshotted memory state may have (5 MiBytes) */
-	private static final int DEFAULT_MAX_STATE_SIZE = 5 * 1024 * 1024;
-	
-	/** The maximal size that the snapshotted memory state may have */
-	private final int maxStateSize;
-
-	/**
-	 * Creates a new memory state backend that accepts states whose serialized forms are
-	 * up to the default state size (5 MB).
-	 */
-	public MemoryStateBackend() {
-		this(DEFAULT_MAX_STATE_SIZE);
-	}
-
-	/**
-	 * Creates a new memory state backend that accepts states whose serialized forms are
-	 * up to the given number of bytes.
-	 * 
-	 * @param maxStateSize The maximal size of the serialized state
-	 */
-	public MemoryStateBackend(int maxStateSize) {
-		this.maxStateSize = maxStateSize;
-	}
-
-	// ------------------------------------------------------------------------
-	//  initialization and cleanup
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void initializeForJob(JobID job) {
-		// nothing to do here
-	}
-
-	@Override
-	public void disposeAllStateForCurrentJob() {
-		// nothing to do here, GC will do it
-	}
-
-	@Override
-	public void close() throws Exception {}
-
-	// ------------------------------------------------------------------------
-	//  State backend operations
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public <K, V> MemHeapKvState<K, V> createKvState(
-			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
-		return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue);
-	}
-	
-	/**
-	 * Serialized the given state into bytes using Java serialization and creates a state handle that
-	 * can re-create that state.
-	 * 
-	 * @param state The state to checkpoint.
-	 * @param checkpointID The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 * @param <S> The type of the state.
-	 * 
-	 * @return A state handle that contains the given state serialized as bytes.
-	 * @throws Exception Thrown, if the serialization fails.
-	 */
-	@Override
-	public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
-			S state, long checkpointID, long timestamp) throws Exception
-	{
-		SerializedStateHandle<S> handle = new SerializedStateHandle<>(state);
-		checkSize(handle.getSizeOfSerializedState(), maxStateSize);
-		return new SerializedStateHandle<S>(state);
-	}
-
-	@Override
-	public CheckpointStateOutputStream createCheckpointStateOutputStream(
-			long checkpointID, long timestamp) throws Exception
-	{
-		return new MemoryCheckpointOutputStream(maxStateSize);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return "MemoryStateBackend (data in heap memory / checkpoints to JobManager)";
-	}
-
-	static void checkSize(int size, int maxSize) throws IOException {
-		if (size > maxSize) {
-			throw new IOException(
-					"Size of the state is larger than the maximum permitted memory-backed state. Size="
-							+ size + " , maxSize=" + maxSize
-							+ " . Consider using a different state backend, like the File System State backend.");
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A CheckpointStateOutputStream that writes into a byte array.
-	 */
-	public static final class MemoryCheckpointOutputStream extends CheckpointStateOutputStream {
-		
-		private final ByteArrayOutputStream os = new ByteArrayOutputStream();
-		
-		private final int maxSize;
-		
-		private boolean closed;
-
-		public MemoryCheckpointOutputStream(int maxSize) {
-			this.maxSize = maxSize;
-		}
-
-		@Override
-		public void write(int b) {
-			os.write(b);
-		}
-
-		@Override
-		public void write(byte[] b, int off, int len) {
-			os.write(b, off, len);
-		}
-
-		// --------------------------------------------------------------------
-
-		@Override
-		public void close() {
-			closed = true;
-			os.reset();
-		}
-
-		@Override
-		public StreamStateHandle closeAndGetHandle() throws IOException {
-			return new ByteStreamStateHandle(closeAndGetBytes());
-		}
-
-		/**
-		 * Closes the stream and returns the byte array containing the stream's data.
-		 * @return The byte array containing the stream's data.
-		 * @throws IOException Thrown if the size of the data exceeds the maximal 
-		 */
-		public byte[] closeAndGetBytes() throws IOException {
-			if (!closed) {
-				checkSize(os.size(), maxSize);
-				byte[] bytes = os.toByteArray();
-				close();
-				return bytes;
-			}
-			else {
-				throw new IllegalStateException("stream has already been closed");
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Static default instance
-	// ------------------------------------------------------------------------
-	
-	/** The default instance of this state backend, using the default maximal state size */
-	private static final MemoryStateBackend DEFAULT_INSTANCE = new MemoryStateBackend();
-
-	/**
-	 * Gets the default instance of this state backend, using the default maximal state size.
-	 * @return The default instance of this state backend.
-	 */
-	public static MemoryStateBackend defaultInstance() {
-		return DEFAULT_INSTANCE;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
deleted file mode 100644
index 163cadd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.memory;
-
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.util.SerializedValue;
-
-import java.io.IOException;
-
-/**
- * A state handle that represents its state in serialized form as bytes.
- *
- * @param <T> The type of state represented by this state handle.
- */
-public class SerializedStateHandle<T> extends SerializedValue<T> implements StateHandle<T> {
-	
-	private static final long serialVersionUID = 4145685722538475769L;
-
-	public SerializedStateHandle(T value) throws IOException {
-		super(value);
-	}
-	
-	@Override
-	public T getState(ClassLoader classLoader) throws Exception {
-		return deserializeValue(classLoader);
-	}
-
-	/**
-	 * Discarding heap-memory backed state is a no-op, so this method does nothing.
-	 */
-	@Override
-	public void discardState() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
index cf8575e..9964760 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.util.MathUtils;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.runtime.operators.Triggerable;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 72a8c25..8c58e29 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -30,17 +30,16 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.StatefulTask;
-import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.StateBackendFactory;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackendFactory;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendFactory;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -493,55 +492,52 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
 	
 	private StateBackend<?> createStateBackend() throws Exception {
 		StateBackend<?> configuredBackend = configuration.getStateBackend(userClassLoader);
-		
+
 		if (configuredBackend != null) {
 			// backend has been configured on the environment
 			LOG.info("Using user-defined state backend: " + configuredBackend);
 			return configuredBackend;
-		}
-		else {
+		} else {
 			// see if we have a backend specified in the configuration
 			Configuration flinkConfig = getEnvironment().getTaskManagerInfo().getConfiguration();
 			String backendName = flinkConfig.getString(ConfigConstants.STATE_BACKEND, null);
-			
+
 			if (backendName == null) {
 				LOG.warn("No state backend has been specified, using default state backend (Memory / JobManager)");
 				backendName = "jobmanager";
 			}
-			
+
 			backendName = backendName.toLowerCase();
 			switch (backendName) {
 				case "jobmanager":
 					LOG.info("State backend is set to heap memory (checkpoint to jobmanager)");
 					return MemoryStateBackend.defaultInstance();
-				
+
 				case "filesystem":
 					FsStateBackend backend = new FsStateBackendFactory().createFromConfig(flinkConfig);
 					LOG.info("State backend is set to heap memory (checkpoints to filesystem \""
-							+ backend.getBasePath() + "\")");
+						+ backend.getBasePath() + "\")");
 					return backend;
-				
+
 				default:
 					try {
 						@SuppressWarnings("rawtypes")
 						Class<? extends StateBackendFactory> clazz =
-								Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class);
+							Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class);
 
 						return (StateBackend<?>) clazz.newInstance();
-					}
-					catch (ClassNotFoundException e) {
+					} catch (ClassNotFoundException e) {
 						throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName);
-					}
-					catch (ClassCastException e) {
+					} catch (ClassCastException e) {
 						throw new IllegalConfigurationException("The class configured under '" +
-								ConfigConstants.STATE_BACKEND + "' is not a valid state backend factory (" +
-								backendName + ')');
-					}
-					catch (Throwable t) {
+							ConfigConstants.STATE_BACKEND + "' is not a valid state backend factory (" +
+							backendName + ')');
+					} catch (Throwable t) {
 						throw new IllegalConfigurationException("Cannot create configured state backend", t);
 					}
 			}
 		}
+	}
 
 	/**
 	 * Registers a timer.

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
index 334fd44..afeabd9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
@@ -19,7 +19,7 @@
 package org.apache.flink.streaming.runtime.tasks;
 
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.KvStateSnapshot;
+import org.apache.flink.runtime.state.KvStateSnapshot;
 
 import java.io.Serializable;
 import java.util.ConcurrentModificationException;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
deleted file mode 100644
index 73100d1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
+++ /dev/null
@@ -1,419 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.FloatSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.filesystem.FileStreamStateHandle;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
-import org.apache.flink.types.StringValue;
-import org.apache.flink.util.OperatingSystem;
-
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.Random;
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-
-public class FileStateBackendTest {
-	
-	@Test
-	public void testSetupAndSerialization() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
-		try {
-			final String backendDir = localFileUri(tempDir);
-			FsStateBackend originalBackend = new FsStateBackend(backendDir);
-			
-			assertFalse(originalBackend.isInitialized());
-			assertEquals(new URI(backendDir), originalBackend.getBasePath().toUri());
-			assertNull(originalBackend.getCheckpointDirectory());
-			
-			// serialize / copy the backend
-			FsStateBackend backend = CommonTestUtils.createCopySerializable(originalBackend);
-			assertFalse(backend.isInitialized());
-			assertEquals(new URI(backendDir), backend.getBasePath().toUri());
-			assertNull(backend.getCheckpointDirectory());
-			
-			// no file operations should be possible right now
-			try {
-				backend.checkpointStateSerializable("exception train rolling in", 2L, System.currentTimeMillis());
-				fail("should fail with an exception");
-			} catch (IllegalStateException e) {
-				// supreme!
-			}
-			
-			backend.initializeForJob(new JobID());
-			assertNotNull(backend.getCheckpointDirectory());
-			
-			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
-			assertTrue(checkpointDir.exists());
-			assertTrue(isDirectoryEmpty(checkpointDir));
-			
-			backend.disposeAllStateForCurrentJob();
-			assertNull(backend.getCheckpointDirectory());
-			
-			assertTrue(isDirectoryEmpty(tempDir));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			deleteDirectorySilently(tempDir);
-		}
-	}
-	
-	@Test
-	public void testSerializableState() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
-		try {
-			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
-			backend.initializeForJob(new JobID());
-
-			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
-
-			String state1 = "dummy state";
-			String state2 = "row row row your boat";
-			Integer state3 = 42;
-			
-			StateHandle<String> handle1 = backend.checkpointStateSerializable(state1, 439568923746L, System.currentTimeMillis());
-			StateHandle<String> handle2 = backend.checkpointStateSerializable(state2, 439568923746L, System.currentTimeMillis());
-			StateHandle<Integer> handle3 = backend.checkpointStateSerializable(state3, 439568923746L, System.currentTimeMillis());
-
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			assertEquals(state1, handle1.getState(getClass().getClassLoader()));
-			handle1.discardState();
-			
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			assertEquals(state2, handle2.getState(getClass().getClassLoader()));
-			handle2.discardState();
-			
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			assertEquals(state3, handle3.getState(getClass().getClassLoader()));
-			handle3.discardState();
-			
-			assertTrue(isDirectoryEmpty(checkpointDir));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			deleteDirectorySilently(tempDir);
-		}
-	}
-
-	@Test
-	public void testStateOutputStream() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
-		try {
-			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
-			backend.initializeForJob(new JobID());
-
-			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
-
-			byte[] state1 = new byte[1274673];
-			byte[] state2 = new byte[1];
-			byte[] state3 = new byte[0];
-			byte[] state4 = new byte[177];
-			
-			Random rnd = new Random();
-			rnd.nextBytes(state1);
-			rnd.nextBytes(state2);
-			rnd.nextBytes(state3);
-			rnd.nextBytes(state4);
-
-			long checkpointId = 97231523452L;
-
-			FsStateBackend.FsCheckpointStateOutputStream stream1 = 
-					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
-			FsStateBackend.FsCheckpointStateOutputStream stream2 =
-					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
-			FsStateBackend.FsCheckpointStateOutputStream stream3 =
-					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
-			
-			stream1.write(state1);
-			stream2.write(state2);
-			stream3.write(state3);
-			
-			FileStreamStateHandle handle1 = stream1.closeAndGetHandle();
-			FileStreamStateHandle handle2 = stream2.closeAndGetHandle();
-			FileStreamStateHandle handle3 = stream3.closeAndGetHandle();
-			
-			// use with try-with-resources
-			StreamStateHandle handle4;
-			try (StateBackend.CheckpointStateOutputStream stream4 =
-					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis())) {
-				stream4.write(state4);
-				handle4 = stream4.closeAndGetHandle();
-			}
-			
-			// close before accessing handle
-			StateBackend.CheckpointStateOutputStream stream5 =
-					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
-			stream5.write(state4);
-			stream5.close();
-			try {
-				stream5.closeAndGetHandle();
-				fail();
-			} catch (IOException e) {
-				// uh-huh
-			}
-			
-			validateBytesInStream(handle1.getState(getClass().getClassLoader()), state1);
-			handle1.discardState();
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			ensureLocalFileDeleted(handle1.getFilePath());
-			
-			validateBytesInStream(handle2.getState(getClass().getClassLoader()), state2);
-			handle2.discardState();
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			ensureLocalFileDeleted(handle2.getFilePath());
-			
-			validateBytesInStream(handle3.getState(getClass().getClassLoader()), state3);
-			handle3.discardState();
-			assertFalse(isDirectoryEmpty(checkpointDir));
-			ensureLocalFileDeleted(handle3.getFilePath());
-			
-			validateBytesInStream(handle4.getState(getClass().getClassLoader()), state4);
-			handle4.discardState();
-			assertTrue(isDirectoryEmpty(checkpointDir));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			deleteDirectorySilently(tempDir);
-		}
-	}
-
-	@Test
-	public void testKeyValueState() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
-		try {
-			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
-			backend.initializeForJob(new JobID());
-
-			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
-
-			KvState<Integer, String, FsStateBackend> kv =
-					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
-
-			assertEquals(0, kv.size());
-
-			// some modifications to the state
-			kv.setCurrentKey(1);
-			assertNull(kv.value());
-			kv.update("1");
-			assertEquals(1, kv.size());
-			kv.setCurrentKey(2);
-			assertNull(kv.value());
-			kv.update("2");
-			assertEquals(2, kv.size());
-			kv.setCurrentKey(1);
-			assertEquals("1", kv.value());
-			assertEquals(2, kv.size());
-
-			// draw a snapshot
-			KvStateSnapshot<Integer, String, FsStateBackend> snapshot1 =
-					kv.shapshot(682375462378L, System.currentTimeMillis());
-
-			// make some more modifications
-			kv.setCurrentKey(1);
-			kv.update("u1");
-			kv.setCurrentKey(2);
-			kv.update("u2");
-			kv.setCurrentKey(3);
-			kv.update("u3");
-
-			// draw another snapshot
-			KvStateSnapshot<Integer, String, FsStateBackend> snapshot2 =
-					kv.shapshot(682375462379L, System.currentTimeMillis());
-
-			// validate the original state
-			assertEquals(3, kv.size());
-			kv.setCurrentKey(1);
-			assertEquals("u1", kv.value());
-			kv.setCurrentKey(2);
-			assertEquals("u2", kv.value());
-			kv.setCurrentKey(3);
-			assertEquals("u3", kv.value());
-
-			// restore the first snapshot and validate it
-			KvState<Integer, String, FsStateBackend> restored1 = snapshot1.restoreState(backend,
-					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
-
-			assertEquals(2, restored1.size());
-			restored1.setCurrentKey(1);
-			assertEquals("1", restored1.value());
-			restored1.setCurrentKey(2);
-			assertEquals("2", restored1.value());
-
-			// restore the first snapshot and validate it
-			KvState<Integer, String, FsStateBackend> restored2 = snapshot2.restoreState(backend,
-					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
-
-			assertEquals(3, restored2.size());
-			restored2.setCurrentKey(1);
-			assertEquals("u1", restored2.value());
-			restored2.setCurrentKey(2);
-			assertEquals("u2", restored2.value());
-			restored2.setCurrentKey(3);
-			assertEquals("u3", restored2.value());
-
-			snapshot1.discardState();
-			assertFalse(isDirectoryEmpty(checkpointDir));
-
-			snapshot2.discardState();
-			assertTrue(isDirectoryEmpty(checkpointDir));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			deleteDirectorySilently(tempDir);
-		}
-	}
-
-	@Test
-	public void testRestoreWithWrongSerializers() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
-		try {
-			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
-			backend.initializeForJob(new JobID());
-
-			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
-			
-			KvState<Integer, String, FsStateBackend> kv =
-					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
-
-			kv.setCurrentKey(1);
-			kv.update("1");
-			kv.setCurrentKey(2);
-			kv.update("2");
-
-			KvStateSnapshot<Integer, String, FsStateBackend> snapshot =
-					kv.shapshot(682375462378L, System.currentTimeMillis());
-
-
-			@SuppressWarnings("unchecked")
-			TypeSerializer<Integer> fakeIntSerializer =
-					(TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
-
-			@SuppressWarnings("unchecked")
-			TypeSerializer<String> fakeStringSerializer =
-					(TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
-
-			try {
-				snapshot.restoreState(backend, fakeIntSerializer,
-						StringSerializer.INSTANCE, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-
-			try {
-				snapshot.restoreState(backend, IntSerializer.INSTANCE,
-						fakeStringSerializer, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-
-			try {
-				snapshot.restoreState(backend, fakeIntSerializer,
-						fakeStringSerializer, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-			
-			snapshot.discardState();
-
-			assertTrue(isDirectoryEmpty(checkpointDir));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			deleteDirectorySilently(tempDir);
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	private static void ensureLocalFileDeleted(Path path) {
-		URI uri = path.toUri();
-		if ("file".equals(uri.getScheme())) {
-			File file = new File(uri.getPath());
-			assertFalse("file not properly deleted", file.exists());
-		}
-		else {
-			throw new IllegalArgumentException("not a local path");
-		}
-	}
-	
-	private static void deleteDirectorySilently(File dir) {
-		try {
-			FileUtils.deleteDirectory(dir);
-		}
-		catch (IOException ignored) {}
-	}
-	
-	private static boolean isDirectoryEmpty(File directory) {
-		String[] nested = directory.list();
-		return  nested == null || nested.length == 0;
-	}
-	
-	private static String localFileUri(File path) {
-		return (OperatingSystem.isWindows() ? "file:/" : "file://") + path.getAbsolutePath();
-	}
-	
-	private static void validateBytesInStream(InputStream is, byte[] data) throws IOException {
-		byte[] holder = new byte[data.length];
-		assertEquals("not enough data", holder.length, is.read(holder));
-		assertEquals("too much data", -1, is.read());
-		assertArrayEquals("wrong data", data, holder);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
deleted file mode 100644
index 3410d09..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.FloatSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
-import org.apache.flink.types.StringValue;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.HashMap;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.state.memory.MemoryStateBackend}.
- */
-public class MemoryStateBackendTest {
-	
-	@Test
-	public void testSerializableState() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend();
-
-			HashMap<String, Integer> state = new HashMap<>();
-			state.put("hey there", 2);
-			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
-			
-			StateHandle<HashMap<String, Integer>> handle = backend.checkpointStateSerializable(state, 12, 459);
-			assertNotNull(handle);
-			
-			HashMap<String, Integer> restored = handle.getState(getClass().getClassLoader());
-			assertEquals(state, restored);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testOversizedState() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend(10);
-
-			HashMap<String, Integer> state = new HashMap<>();
-			state.put("hey there", 2);
-			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
-
-			try {
-				backend.checkpointStateSerializable(state, 12, 459);
-				fail("this should cause an exception");
-			}
-			catch (IOException e) {
-				// now darling, isn't that exactly what we wanted?
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testStateStream() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend();
-
-			HashMap<String, Integer> state = new HashMap<>();
-			state.put("hey there", 2);
-			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
-
-			StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
-			ObjectOutputStream oos = new ObjectOutputStream(os);
-			oos.writeObject(state);
-			oos.flush();
-			StreamStateHandle handle = os.closeAndGetHandle();
-			
-			assertNotNull(handle);
-
-			ObjectInputStream ois = new ObjectInputStream(handle.getState(getClass().getClassLoader()));
-			assertEquals(state, ois.readObject());
-			assertTrue(ois.available() <= 0);
-			ois.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testOversizedStateStream() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend(10);
-
-			HashMap<String, Integer> state = new HashMap<>();
-			state.put("hey there", 2);
-			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
-
-			StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
-			ObjectOutputStream oos = new ObjectOutputStream(os);
-			
-			try {
-				oos.writeObject(state);
-				oos.flush();
-				os.closeAndGetHandle();
-				fail("this should cause an exception");
-			}
-			catch (IOException e) {
-				// oh boy! what an exception!
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testKeyValueState() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend();
-			
-			KvState<Integer, String, MemoryStateBackend> kv = 
-					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
-			
-			assertEquals(0, kv.size());
-			
-			// some modifications to the state
-			kv.setCurrentKey(1);
-			assertNull(kv.value());
-			kv.update("1");
-			assertEquals(1, kv.size());
-			kv.setCurrentKey(2);
-			assertNull(kv.value());
-			kv.update("2");
-			assertEquals(2, kv.size());
-			kv.setCurrentKey(1);
-			assertEquals("1", kv.value());
-			assertEquals(2, kv.size());
-			
-			// draw a snapshot
-			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot1 = 
-					kv.shapshot(682375462378L, System.currentTimeMillis());
-			
-			// make some more modifications
-			kv.setCurrentKey(1);
-			kv.update("u1");
-			kv.setCurrentKey(2);
-			kv.update("u2");
-			kv.setCurrentKey(3);
-			kv.update("u3");
-
-			// draw another snapshot
-			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot2 =
-					kv.shapshot(682375462379L, System.currentTimeMillis());
-			
-			// validate the original state
-			assertEquals(3, kv.size());
-			kv.setCurrentKey(1);
-			assertEquals("u1", kv.value());
-			kv.setCurrentKey(2);
-			assertEquals("u2", kv.value());
-			kv.setCurrentKey(3);
-			assertEquals("u3", kv.value());
-			
-			// restore the first snapshot and validate it
-			KvState<Integer, String, MemoryStateBackend> restored1 = snapshot1.restoreState(backend, 
-							IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
-
-			assertEquals(2, restored1.size());
-			restored1.setCurrentKey(1);
-			assertEquals("1", restored1.value());
-			restored1.setCurrentKey(2);
-			assertEquals("2", restored1.value());
-
-			// restore the first snapshot and validate it
-			KvState<Integer, String, MemoryStateBackend> restored2 = snapshot2.restoreState(backend,
-					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
-
-			assertEquals(3, restored2.size());
-			restored2.setCurrentKey(1);
-			assertEquals("u1", restored2.value());
-			restored2.setCurrentKey(2);
-			assertEquals("u2", restored2.value());
-			restored2.setCurrentKey(3);
-			assertEquals("u3", restored2.value());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testRestoreWithWrongSerializers() {
-		try {
-			MemoryStateBackend backend = new MemoryStateBackend();
-			KvState<Integer, String, MemoryStateBackend> kv =
-					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
-			
-			kv.setCurrentKey(1);
-			kv.update("1");
-			kv.setCurrentKey(2);
-			kv.update("2");
-			
-			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot =
-					kv.shapshot(682375462378L, System.currentTimeMillis());
-
-
-			@SuppressWarnings("unchecked")
-			TypeSerializer<Integer> fakeIntSerializer = 
-					(TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
-
-			@SuppressWarnings("unchecked")
-			TypeSerializer<String> fakeStringSerializer = 
-					(TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
-
-			try {
-				snapshot.restoreState(backend, fakeIntSerializer,
-						StringSerializer.INSTANCE, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-
-			try {
-				snapshot.restoreState(backend, IntSerializer.INSTANCE,
-						fakeStringSerializer, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-
-			try {
-				snapshot.restoreState(backend, fakeIntSerializer,
-						fakeStringSerializer, null, getClass().getClassLoader());
-				fail("should recognize wrong serializers");
-			} catch (IllegalArgumentException e) {
-				// expected
-			} catch (Exception e) {
-				fail("wrong exception");
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index dd76a67..ad3c838 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -28,8 +28,8 @@ import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index ab8e551..4bd260f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -28,8 +28,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
index 81d3a69..0c708c6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
@@ -36,8 +36,8 @@ import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index b83feca..01f95bc 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -30,8 +30,8 @@ import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;


[40/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/js/vendor.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/vendor.js b/flink-runtime-web/src/main/resources/web/js/vendor.js
deleted file mode 100644
index 52bc81c..0000000
--- a/flink-runtime-web/src/main/resources/web/js/vendor.js
+++ /dev/null
@@ -1,81087 +0,0 @@
-/*!
- * jQuery JavaScript Library v2.1.4
- * http://jquery.com/
- *
- * Includes Sizzle.js
- * http://sizzlejs.com/
- *
- * Copyright 2005, 2014 jQuery Foundation, Inc. and other contributors
- * Released under the MIT license
- * http://jquery.org/license
- *
- * Date: 2015-04-28T16:01Z
- */
-
-(function( global, factory ) {
-
-	if ( typeof module === "object" && typeof module.exports === "object" ) {
-		// For CommonJS and CommonJS-like environments where a proper `window`
-		// is present, execute the factory and get jQuery.
-		// For environments that do not have a `window` with a `document`
-		// (such as Node.js), expose a factory as module.exports.
-		// This accentuates the need for the creation of a real `window`.
-		// e.g. var jQuery = require("jquery")(window);
-		// See ticket #14549 for more info.
-		module.exports = global.document ?
-			factory( global, true ) :
-			function( w ) {
-				if ( !w.document ) {
-					throw new Error( "jQuery requires a window with a document" );
-				}
-				return factory( w );
-			};
-	} else {
-		factory( global );
-	}
-
-// Pass this if window is not defined yet
-}(typeof window !== "undefined" ? window : this, function( window, noGlobal ) {
-
-// Support: Firefox 18+
-// Can't be in strict mode, several libs including ASP.NET trace
-// the stack via arguments.caller.callee and Firefox dies if
-// you try to trace through "use strict" call chains. (#13335)
-//
-
-var arr = [];
-
-var slice = arr.slice;
-
-var concat = arr.concat;
-
-var push = arr.push;
-
-var indexOf = arr.indexOf;
-
-var class2type = {};
-
-var toString = class2type.toString;
-
-var hasOwn = class2type.hasOwnProperty;
-
-var support = {};
-
-
-
-var
-	// Use the correct document accordingly with window argument (sandbox)
-	document = window.document,
-
-	version = "2.1.4",
-
-	// Define a local copy of jQuery
-	jQuery = function( selector, context ) {
-		// The jQuery object is actually just the init constructor 'enhanced'
-		// Need init if jQuery is called (just allow error to be thrown if not included)
-		return new jQuery.fn.init( selector, context );
-	},
-
-	// Support: Android<4.1
-	// Make sure we trim BOM and NBSP
-	rtrim = /^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,
-
-	// Matches dashed string for camelizing
-	rmsPrefix = /^-ms-/,
-	rdashAlpha = /-([\da-z])/gi,
-
-	// Used by jQuery.camelCase as callback to replace()
-	fcamelCase = function( all, letter ) {
-		return letter.toUpperCase();
-	};
-
-jQuery.fn = jQuery.prototype = {
-	// The current version of jQuery being used
-	jquery: version,
-
-	constructor: jQuery,
-
-	// Start with an empty selector
-	selector: "",
-
-	// The default length of a jQuery object is 0
-	length: 0,
-
-	toArray: function() {
-		return slice.call( this );
-	},
-
-	// Get the Nth element in the matched element set OR
-	// Get the whole matched element set as a clean array
-	get: function( num ) {
-		return num != null ?
-
-			// Return just the one element from the set
-			( num < 0 ? this[ num + this.length ] : this[ num ] ) :
-
-			// Return all the elements in a clean array
-			slice.call( this );
-	},
-
-	// Take an array of elements and push it onto the stack
-	// (returning the new matched element set)
-	pushStack: function( elems ) {
-
-		// Build a new jQuery matched element set
-		var ret = jQuery.merge( this.constructor(), elems );
-
-		// Add the old object onto the stack (as a reference)
-		ret.prevObject = this;
-		ret.context = this.context;
-
-		// Return the newly-formed element set
-		return ret;
-	},
-
-	// Execute a callback for every element in the matched set.
-	// (You can seed the arguments with an array of args, but this is
-	// only used internally.)
-	each: function( callback, args ) {
-		return jQuery.each( this, callback, args );
-	},
-
-	map: function( callback ) {
-		return this.pushStack( jQuery.map(this, function( elem, i ) {
-			return callback.call( elem, i, elem );
-		}));
-	},
-
-	slice: function() {
-		return this.pushStack( slice.apply( this, arguments ) );
-	},
-
-	first: function() {
-		return this.eq( 0 );
-	},
-
-	last: function() {
-		return this.eq( -1 );
-	},
-
-	eq: function( i ) {
-		var len = this.length,
-			j = +i + ( i < 0 ? len : 0 );
-		return this.pushStack( j >= 0 && j < len ? [ this[j] ] : [] );
-	},
-
-	end: function() {
-		return this.prevObject || this.constructor(null);
-	},
-
-	// For internal use only.
-	// Behaves like an Array's method, not like a jQuery method.
-	push: push,
-	sort: arr.sort,
-	splice: arr.splice
-};
-
-jQuery.extend = jQuery.fn.extend = function() {
-	var options, name, src, copy, copyIsArray, clone,
-		target = arguments[0] || {},
-		i = 1,
-		length = arguments.length,
-		deep = false;
-
-	// Handle a deep copy situation
-	if ( typeof target === "boolean" ) {
-		deep = target;
-
-		// Skip the boolean and the target
-		target = arguments[ i ] || {};
-		i++;
-	}
-
-	// Handle case when target is a string or something (possible in deep copy)
-	if ( typeof target !== "object" && !jQuery.isFunction(target) ) {
-		target = {};
-	}
-
-	// Extend jQuery itself if only one argument is passed
-	if ( i === length ) {
-		target = this;
-		i--;
-	}
-
-	for ( ; i < length; i++ ) {
-		// Only deal with non-null/undefined values
-		if ( (options = arguments[ i ]) != null ) {
-			// Extend the base object
-			for ( name in options ) {
-				src = target[ name ];
-				copy = options[ name ];
-
-				// Prevent never-ending loop
-				if ( target === copy ) {
-					continue;
-				}
-
-				// Recurse if we're merging plain objects or arrays
-				if ( deep && copy && ( jQuery.isPlainObject(copy) || (copyIsArray = jQuery.isArray(copy)) ) ) {
-					if ( copyIsArray ) {
-						copyIsArray = false;
-						clone = src && jQuery.isArray(src) ? src : [];
-
-					} else {
-						clone = src && jQuery.isPlainObject(src) ? src : {};
-					}
-
-					// Never move original objects, clone them
-					target[ name ] = jQuery.extend( deep, clone, copy );
-
-				// Don't bring in undefined values
-				} else if ( copy !== undefined ) {
-					target[ name ] = copy;
-				}
-			}
-		}
-	}
-
-	// Return the modified object
-	return target;
-};
-
-jQuery.extend({
-	// Unique for each copy of jQuery on the page
-	expando: "jQuery" + ( version + Math.random() ).replace( /\D/g, "" ),
-
-	// Assume jQuery is ready without the ready module
-	isReady: true,
-
-	error: function( msg ) {
-		throw new Error( msg );
-	},
-
-	noop: function() {},
-
-	isFunction: function( obj ) {
-		return jQuery.type(obj) === "function";
-	},
-
-	isArray: Array.isArray,
-
-	isWindow: function( obj ) {
-		return obj != null && obj === obj.window;
-	},
-
-	isNumeric: function( obj ) {
-		// parseFloat NaNs numeric-cast false positives (null|true|false|"")
-		// ...but misinterprets leading-number strings, particularly hex literals ("0x...")
-		// subtraction forces infinities to NaN
-		// adding 1 corrects loss of precision from parseFloat (#15100)
-		return !jQuery.isArray( obj ) && (obj - parseFloat( obj ) + 1) >= 0;
-	},
-
-	isPlainObject: function( obj ) {
-		// Not plain objects:
-		// - Any object or value whose internal [[Class]] property is not "[object Object]"
-		// - DOM nodes
-		// - window
-		if ( jQuery.type( obj ) !== "object" || obj.nodeType || jQuery.isWindow( obj ) ) {
-			return false;
-		}
-
-		if ( obj.constructor &&
-				!hasOwn.call( obj.constructor.prototype, "isPrototypeOf" ) ) {
-			return false;
-		}
-
-		// If the function hasn't returned already, we're confident that
-		// |obj| is a plain object, created by {} or constructed with new Object
-		return true;
-	},
-
-	isEmptyObject: function( obj ) {
-		var name;
-		for ( name in obj ) {
-			return false;
-		}
-		return true;
-	},
-
-	type: function( obj ) {
-		if ( obj == null ) {
-			return obj + "";
-		}
-		// Support: Android<4.0, iOS<6 (functionish RegExp)
-		return typeof obj === "object" || typeof obj === "function" ?
-			class2type[ toString.call(obj) ] || "object" :
-			typeof obj;
-	},
-
-	// Evaluates a script in a global context
-	globalEval: function( code ) {
-		var script,
-			indirect = eval;
-
-		code = jQuery.trim( code );
-
-		if ( code ) {
-			// If the code includes a valid, prologue position
-			// strict mode pragma, execute code by injecting a
-			// script tag into the document.
-			if ( code.indexOf("use strict") === 1 ) {
-				script = document.createElement("script");
-				script.text = code;
-				document.head.appendChild( script ).parentNode.removeChild( script );
-			} else {
-			// Otherwise, avoid the DOM node creation, insertion
-			// and removal by using an indirect global eval
-				indirect( code );
-			}
-		}
-	},
-
-	// Convert dashed to camelCase; used by the css and data modules
-	// Support: IE9-11+
-	// Microsoft forgot to hump their vendor prefix (#9572)
-	camelCase: function( string ) {
-		return string.replace( rmsPrefix, "ms-" ).replace( rdashAlpha, fcamelCase );
-	},
-
-	nodeName: function( elem, name ) {
-		return elem.nodeName && elem.nodeName.toLowerCase() === name.toLowerCase();
-	},
-
-	// args is for internal usage only
-	each: function( obj, callback, args ) {
-		var value,
-			i = 0,
-			length = obj.length,
-			isArray = isArraylike( obj );
-
-		if ( args ) {
-			if ( isArray ) {
-				for ( ; i < length; i++ ) {
-					value = callback.apply( obj[ i ], args );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			} else {
-				for ( i in obj ) {
-					value = callback.apply( obj[ i ], args );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			}
-
-		// A special, fast, case for the most common use of each
-		} else {
-			if ( isArray ) {
-				for ( ; i < length; i++ ) {
-					value = callback.call( obj[ i ], i, obj[ i ] );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			} else {
-				for ( i in obj ) {
-					value = callback.call( obj[ i ], i, obj[ i ] );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			}
-		}
-
-		return obj;
-	},
-
-	// Support: Android<4.1
-	trim: function( text ) {
-		return text == null ?
-			"" :
-			( text + "" ).replace( rtrim, "" );
-	},
-
-	// results is for internal usage only
-	makeArray: function( arr, results ) {
-		var ret = results || [];
-
-		if ( arr != null ) {
-			if ( isArraylike( Object(arr) ) ) {
-				jQuery.merge( ret,
-					typeof arr === "string" ?
-					[ arr ] : arr
-				);
-			} else {
-				push.call( ret, arr );
-			}
-		}
-
-		return ret;
-	},
-
-	inArray: function( elem, arr, i ) {
-		return arr == null ? -1 : indexOf.call( arr, elem, i );
-	},
-
-	merge: function( first, second ) {
-		var len = +second.length,
-			j = 0,
-			i = first.length;
-
-		for ( ; j < len; j++ ) {
-			first[ i++ ] = second[ j ];
-		}
-
-		first.length = i;
-
-		return first;
-	},
-
-	grep: function( elems, callback, invert ) {
-		var callbackInverse,
-			matches = [],
-			i = 0,
-			length = elems.length,
-			callbackExpect = !invert;
-
-		// Go through the array, only saving the items
-		// that pass the validator function
-		for ( ; i < length; i++ ) {
-			callbackInverse = !callback( elems[ i ], i );
-			if ( callbackInverse !== callbackExpect ) {
-				matches.push( elems[ i ] );
-			}
-		}
-
-		return matches;
-	},
-
-	// arg is for internal usage only
-	map: function( elems, callback, arg ) {
-		var value,
-			i = 0,
-			length = elems.length,
-			isArray = isArraylike( elems ),
-			ret = [];
-
-		// Go through the array, translating each of the items to their new values
-		if ( isArray ) {
-			for ( ; i < length; i++ ) {
-				value = callback( elems[ i ], i, arg );
-
-				if ( value != null ) {
-					ret.push( value );
-				}
-			}
-
-		// Go through every key on the object,
-		} else {
-			for ( i in elems ) {
-				value = callback( elems[ i ], i, arg );
-
-				if ( value != null ) {
-					ret.push( value );
-				}
-			}
-		}
-
-		// Flatten any nested arrays
-		return concat.apply( [], ret );
-	},
-
-	// A global GUID counter for objects
-	guid: 1,
-
-	// Bind a function to a context, optionally partially applying any
-	// arguments.
-	proxy: function( fn, context ) {
-		var tmp, args, proxy;
-
-		if ( typeof context === "string" ) {
-			tmp = fn[ context ];
-			context = fn;
-			fn = tmp;
-		}
-
-		// Quick check to determine if target is callable, in the spec
-		// this throws a TypeError, but we will just return undefined.
-		if ( !jQuery.isFunction( fn ) ) {
-			return undefined;
-		}
-
-		// Simulated bind
-		args = slice.call( arguments, 2 );
-		proxy = function() {
-			return fn.apply( context || this, args.concat( slice.call( arguments ) ) );
-		};
-
-		// Set the guid of unique handler to the same of original handler, so it can be removed
-		proxy.guid = fn.guid = fn.guid || jQuery.guid++;
-
-		return proxy;
-	},
-
-	now: Date.now,
-
-	// jQuery.support is not used in Core but other projects attach their
-	// properties to it so it needs to exist.
-	support: support
-});
-
-// Populate the class2type map
-jQuery.each("Boolean Number String Function Array Date RegExp Object Error".split(" "), function(i, name) {
-	class2type[ "[object " + name + "]" ] = name.toLowerCase();
-});
-
-function isArraylike( obj ) {
-
-	// Support: iOS 8.2 (not reproducible in simulator)
-	// `in` check used to prevent JIT error (gh-2145)
-	// hasOwn isn't used here due to false negatives
-	// regarding Nodelist length in IE
-	var length = "length" in obj && obj.length,
-		type = jQuery.type( obj );
-
-	if ( type === "function" || jQuery.isWindow( obj ) ) {
-		return false;
-	}
-
-	if ( obj.nodeType === 1 && length ) {
-		return true;
-	}
-
-	return type === "array" || length === 0 ||
-		typeof length === "number" && length > 0 && ( length - 1 ) in obj;
-}
-var Sizzle =
-/*!
- * Sizzle CSS Selector Engine v2.2.0-pre
- * http://sizzlejs.com/
- *
- * Copyright 2008, 2014 jQuery Foundation, Inc. and other contributors
- * Released under the MIT license
- * http://jquery.org/license
- *
- * Date: 2014-12-16
- */
-(function( window ) {
-
-var i,
-	support,
-	Expr,
-	getText,
-	isXML,
-	tokenize,
-	compile,
-	select,
-	outermostContext,
-	sortInput,
-	hasDuplicate,
-
-	// Local document vars
-	setDocument,
-	document,
-	docElem,
-	documentIsHTML,
-	rbuggyQSA,
-	rbuggyMatches,
-	matches,
-	contains,
-
-	// Instance-specific data
-	expando = "sizzle" + 1 * new Date(),
-	preferredDoc = window.document,
-	dirruns = 0,
-	done = 0,
-	classCache = createCache(),
-	tokenCache = createCache(),
-	compilerCache = createCache(),
-	sortOrder = function( a, b ) {
-		if ( a === b ) {
-			hasDuplicate = true;
-		}
-		return 0;
-	},
-
-	// General-purpose constants
-	MAX_NEGATIVE = 1 << 31,
-
-	// Instance methods
-	hasOwn = ({}).hasOwnProperty,
-	arr = [],
-	pop = arr.pop,
-	push_native = arr.push,
-	push = arr.push,
-	slice = arr.slice,
-	// Use a stripped-down indexOf as it's faster than native
-	// http://jsperf.com/thor-indexof-vs-for/5
-	indexOf = function( list, elem ) {
-		var i = 0,
-			len = list.length;
-		for ( ; i < len; i++ ) {
-			if ( list[i] === elem ) {
-				return i;
-			}
-		}
-		return -1;
-	},
-
-	booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",
-
-	// Regular expressions
-
-	// Whitespace characters http://www.w3.org/TR/css3-selectors/#whitespace
-	whitespace = "[\\x20\\t\\r\\n\\f]",
-	// http://www.w3.org/TR/css3-syntax/#characters
-	characterEncoding = "(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",
-
-	// Loosely modeled on CSS identifier characters
-	// An unquoted value should be a CSS identifier http://www.w3.org/TR/css3-selectors/#attribute-selectors
-	// Proper syntax: http://www.w3.org/TR/CSS21/syndata.html#value-def-identifier
-	identifier = characterEncoding.replace( "w", "w#" ),
-
-	// Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
-	attributes = "\\[" + whitespace + "*(" + characterEncoding + ")(?:" + whitespace +
-		// Operator (capture 2)
-		"*([*^$|!~]?=)" + whitespace +
-		// "Attribute values must be CSS identifiers [capture 5] or strings [capture 3 or capture 4]"
-		"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + whitespace +
-		"*\\]",
-
-	pseudos = ":(" + characterEncoding + ")(?:\\((" +
-		// To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
-		// 1. quoted (capture 3; capture 4 or capture 5)
-		"('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
-		// 2. simple (capture 6)
-		"((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
-		// 3. anything else (capture 2)
-		".*" +
-		")\\)|)",
-
-	// Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
-	rwhitespace = new RegExp( whitespace + "+", "g" ),
-	rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + whitespace + "+$", "g" ),
-
-	rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
-	rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + "*" ),
-
-	rattributeQuotes = new RegExp( "=" + whitespace + "*([^\\]'\"]*?)" + whitespace + "*\\]", "g" ),
-
-	rpseudo = new RegExp( pseudos ),
-	ridentifier = new RegExp( "^" + identifier + "$" ),
-
-	matchExpr = {
-		"ID": new RegExp( "^#(" + characterEncoding + ")" ),
-		"CLASS": new RegExp( "^\\.(" + characterEncoding + ")" ),
-		"TAG": new RegExp( "^(" + characterEncoding.replace( "w", "w*" ) + ")" ),
-		"ATTR": new RegExp( "^" + attributes ),
-		"PSEUDO": new RegExp( "^" + pseudos ),
-		"CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + whitespace +
-			"*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + whitespace +
-			"*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
-		"bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
-		// For use in libraries implementing .is()
-		// We use this for POS matching in `select`
-		"needsContext": new RegExp( "^" + whitespace + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" +
-			whitespace + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
-	},
-
-	rinputs = /^(?:input|select|textarea|button)$/i,
-	rheader = /^h\d$/i,
-
-	rnative = /^[^{]+\{\s*\[native \w/,
-
-	// Easily-parseable/retrievable ID or TAG or CLASS selectors
-	rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
-
-	rsibling = /[+~]/,
-	rescape = /'|\\/g,
-
-	// CSS escapes http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
-	runescape = new RegExp( "\\\\([\\da-f]{1,6}" + whitespace + "?|(" + whitespace + ")|.)", "ig" ),
-	funescape = function( _, escaped, escapedWhitespace ) {
-		var high = "0x" + escaped - 0x10000;
-		// NaN means non-codepoint
-		// Support: Firefox<24
-		// Workaround erroneous numeric interpretation of +"0x"
-		return high !== high || escapedWhitespace ?
-			escaped :
-			high < 0 ?
-				// BMP codepoint
-				String.fromCharCode( high + 0x10000 ) :
-				// Supplemental Plane codepoint (surrogate pair)
-				String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
-	},
-
-	// Used for iframes
-	// See setDocument()
-	// Removing the function wrapper causes a "Permission Denied"
-	// error in IE
-	unloadHandler = function() {
-		setDocument();
-	};
-
-// Optimize for push.apply( _, NodeList )
-try {
-	push.apply(
-		(arr = slice.call( preferredDoc.childNodes )),
-		preferredDoc.childNodes
-	);
-	// Support: Android<4.0
-	// Detect silently failing push.apply
-	arr[ preferredDoc.childNodes.length ].nodeType;
-} catch ( e ) {
-	push = { apply: arr.length ?
-
-		// Leverage slice if possible
-		function( target, els ) {
-			push_native.apply( target, slice.call(els) );
-		} :
-
-		// Support: IE<9
-		// Otherwise append directly
-		function( target, els ) {
-			var j = target.length,
-				i = 0;
-			// Can't trust NodeList.length
-			while ( (target[j++] = els[i++]) ) {}
-			target.length = j - 1;
-		}
-	};
-}
-
-function Sizzle( selector, context, results, seed ) {
-	var match, elem, m, nodeType,
-		// QSA vars
-		i, groups, old, nid, newContext, newSelector;
-
-	if ( ( context ? context.ownerDocument || context : preferredDoc ) !== document ) {
-		setDocument( context );
-	}
-
-	context = context || document;
-	results = results || [];
-	nodeType = context.nodeType;
-
-	if ( typeof selector !== "string" || !selector ||
-		nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
-
-		return results;
-	}
-
-	if ( !seed && documentIsHTML ) {
-
-		// Try to shortcut find operations when possible (e.g., not under DocumentFragment)
-		if ( nodeType !== 11 && (match = rquickExpr.exec( selector )) ) {
-			// Speed-up: Sizzle("#ID")
-			if ( (m = match[1]) ) {
-				if ( nodeType === 9 ) {
-					elem = context.getElementById( m );
-					// Check parentNode to catch when Blackberry 4.6 returns
-					// nodes that are no longer in the document (jQuery #6963)
-					if ( elem && elem.parentNode ) {
-						// Handle the case where IE, Opera, and Webkit return items
-						// by name instead of ID
-						if ( elem.id === m ) {
-							results.push( elem );
-							return results;
-						}
-					} else {
-						return results;
-					}
-				} else {
-					// Context is not a document
-					if ( context.ownerDocument && (elem = context.ownerDocument.getElementById( m )) &&
-						contains( context, elem ) && elem.id === m ) {
-						results.push( elem );
-						return results;
-					}
-				}
-
-			// Speed-up: Sizzle("TAG")
-			} else if ( match[2] ) {
-				push.apply( results, context.getElementsByTagName( selector ) );
-				return results;
-
-			// Speed-up: Sizzle(".CLASS")
-			} else if ( (m = match[3]) && support.getElementsByClassName ) {
-				push.apply( results, context.getElementsByClassName( m ) );
-				return results;
-			}
-		}
-
-		// QSA path
-		if ( support.qsa && (!rbuggyQSA || !rbuggyQSA.test( selector )) ) {
-			nid = old = expando;
-			newContext = context;
-			newSelector = nodeType !== 1 && selector;
-
-			// qSA works strangely on Element-rooted queries
-			// We can work around this by specifying an extra ID on the root
-			// and working up from there (Thanks to Andrew Dupont for the technique)
-			// IE 8 doesn't work on object elements
-			if ( nodeType === 1 && context.nodeName.toLowerCase() !== "object" ) {
-				groups = tokenize( selector );
-
-				if ( (old = context.getAttribute("id")) ) {
-					nid = old.replace( rescape, "\\$&" );
-				} else {
-					context.setAttribute( "id", nid );
-				}
-				nid = "[id='" + nid + "'] ";
-
-				i = groups.length;
-				while ( i-- ) {
-					groups[i] = nid + toSelector( groups[i] );
-				}
-				newContext = rsibling.test( selector ) && testContext( context.parentNode ) || context;
-				newSelector = groups.join(",");
-			}
-
-			if ( newSelector ) {
-				try {
-					push.apply( results,
-						newContext.querySelectorAll( newSelector )
-					);
-					return results;
-				} catch(qsaError) {
-				} finally {
-					if ( !old ) {
-						context.removeAttribute("id");
-					}
-				}
-			}
-		}
-	}
-
-	// All others
-	return select( selector.replace( rtrim, "$1" ), context, results, seed );
-}
-
-/**
- * Create key-value caches of limited size
- * @returns {Function(string, Object)} Returns the Object data after storing it on itself with
- *	property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
- *	deleting the oldest entry
- */
-function createCache() {
-	var keys = [];
-
-	function cache( key, value ) {
-		// Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
-		if ( keys.push( key + " " ) > Expr.cacheLength ) {
-			// Only keep the most recent entries
-			delete cache[ keys.shift() ];
-		}
-		return (cache[ key + " " ] = value);
-	}
-	return cache;
-}
-
-/**
- * Mark a function for special use by Sizzle
- * @param {Function} fn The function to mark
- */
-function markFunction( fn ) {
-	fn[ expando ] = true;
-	return fn;
-}
-
-/**
- * Support testing using an element
- * @param {Function} fn Passed the created div and expects a boolean result
- */
-function assert( fn ) {
-	var div = document.createElement("div");
-
-	try {
-		return !!fn( div );
-	} catch (e) {
-		return false;
-	} finally {
-		// Remove from its parent by default
-		if ( div.parentNode ) {
-			div.parentNode.removeChild( div );
-		}
-		// release memory in IE
-		div = null;
-	}
-}
-
-/**
- * Adds the same handler for all of the specified attrs
- * @param {String} attrs Pipe-separated list of attributes
- * @param {Function} handler The method that will be applied
- */
-function addHandle( attrs, handler ) {
-	var arr = attrs.split("|"),
-		i = attrs.length;
-
-	while ( i-- ) {
-		Expr.attrHandle[ arr[i] ] = handler;
-	}
-}
-
-/**
- * Checks document order of two siblings
- * @param {Element} a
- * @param {Element} b
- * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
- */
-function siblingCheck( a, b ) {
-	var cur = b && a,
-		diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
-			( ~b.sourceIndex || MAX_NEGATIVE ) -
-			( ~a.sourceIndex || MAX_NEGATIVE );
-
-	// Use IE sourceIndex if available on both nodes
-	if ( diff ) {
-		return diff;
-	}
-
-	// Check if b follows a
-	if ( cur ) {
-		while ( (cur = cur.nextSibling) ) {
-			if ( cur === b ) {
-				return -1;
-			}
-		}
-	}
-
-	return a ? 1 : -1;
-}
-
-/**
- * Returns a function to use in pseudos for input types
- * @param {String} type
- */
-function createInputPseudo( type ) {
-	return function( elem ) {
-		var name = elem.nodeName.toLowerCase();
-		return name === "input" && elem.type === type;
-	};
-}
-
-/**
- * Returns a function to use in pseudos for buttons
- * @param {String} type
- */
-function createButtonPseudo( type ) {
-	return function( elem ) {
-		var name = elem.nodeName.toLowerCase();
-		return (name === "input" || name === "button") && elem.type === type;
-	};
-}
-
-/**
- * Returns a function to use in pseudos for positionals
- * @param {Function} fn
- */
-function createPositionalPseudo( fn ) {
-	return markFunction(function( argument ) {
-		argument = +argument;
-		return markFunction(function( seed, matches ) {
-			var j,
-				matchIndexes = fn( [], seed.length, argument ),
-				i = matchIndexes.length;
-
-			// Match elements found at the specified indexes
-			while ( i-- ) {
-				if ( seed[ (j = matchIndexes[i]) ] ) {
-					seed[j] = !(matches[j] = seed[j]);
-				}
-			}
-		});
-	});
-}
-
-/**
- * Checks a node for validity as a Sizzle context
- * @param {Element|Object=} context
- * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
- */
-function testContext( context ) {
-	return context && typeof context.getElementsByTagName !== "undefined" && context;
-}
-
-// Expose support vars for convenience
-support = Sizzle.support = {};
-
-/**
- * Detects XML nodes
- * @param {Element|Object} elem An element or a document
- * @returns {Boolean} True iff elem is a non-HTML XML node
- */
-isXML = Sizzle.isXML = function( elem ) {
-	// documentElement is verified for cases where it doesn't yet exist
-	// (such as loading iframes in IE - #4833)
-	var documentElement = elem && (elem.ownerDocument || elem).documentElement;
-	return documentElement ? documentElement.nodeName !== "HTML" : false;
-};
-
-/**
- * Sets document-related variables once based on the current document
- * @param {Element|Object} [doc] An element or document object to use to set the document
- * @returns {Object} Returns the current document
- */
-setDocument = Sizzle.setDocument = function( node ) {
-	var hasCompare, parent,
-		doc = node ? node.ownerDocument || node : preferredDoc;
-
-	// If no document and documentElement is available, return
-	if ( doc === document || doc.nodeType !== 9 || !doc.documentElement ) {
-		return document;
-	}
-
-	// Set our document
-	document = doc;
-	docElem = doc.documentElement;
-	parent = doc.defaultView;
-
-	// Support: IE>8
-	// If iframe document is assigned to "document" variable and if iframe has been reloaded,
-	// IE will throw "permission denied" error when accessing "document" variable, see jQuery #13936
-	// IE6-8 do not support the defaultView property so parent will be undefined
-	if ( parent && parent !== parent.top ) {
-		// IE11 does not have attachEvent, so all must suffer
-		if ( parent.addEventListener ) {
-			parent.addEventListener( "unload", unloadHandler, false );
-		} else if ( parent.attachEvent ) {
-			parent.attachEvent( "onunload", unloadHandler );
-		}
-	}
-
-	/* Support tests
-	---------------------------------------------------------------------- */
-	documentIsHTML = !isXML( doc );
-
-	/* Attributes
-	---------------------------------------------------------------------- */
-
-	// Support: IE<8
-	// Verify that getAttribute really returns attributes and not properties
-	// (excepting IE8 booleans)
-	support.attributes = assert(function( div ) {
-		div.className = "i";
-		return !div.getAttribute("className");
-	});
-
-	/* getElement(s)By*
-	---------------------------------------------------------------------- */
-
-	// Check if getElementsByTagName("*") returns only elements
-	support.getElementsByTagName = assert(function( div ) {
-		div.appendChild( doc.createComment("") );
-		return !div.getElementsByTagName("*").length;
-	});
-
-	// Support: IE<9
-	support.getElementsByClassName = rnative.test( doc.getElementsByClassName );
-
-	// Support: IE<10
-	// Check if getElementById returns elements by name
-	// The broken getElementById methods don't pick up programatically-set names,
-	// so use a roundabout getElementsByName test
-	support.getById = assert(function( div ) {
-		docElem.appendChild( div ).id = expando;
-		return !doc.getElementsByName || !doc.getElementsByName( expando ).length;
-	});
-
-	// ID find and filter
-	if ( support.getById ) {
-		Expr.find["ID"] = function( id, context ) {
-			if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
-				var m = context.getElementById( id );
-				// Check parentNode to catch when Blackberry 4.6 returns
-				// nodes that are no longer in the document #6963
-				return m && m.parentNode ? [ m ] : [];
-			}
-		};
-		Expr.filter["ID"] = function( id ) {
-			var attrId = id.replace( runescape, funescape );
-			return function( elem ) {
-				return elem.getAttribute("id") === attrId;
-			};
-		};
-	} else {
-		// Support: IE6/7
-		// getElementById is not reliable as a find shortcut
-		delete Expr.find["ID"];
-
-		Expr.filter["ID"] =  function( id ) {
-			var attrId = id.replace( runescape, funescape );
-			return function( elem ) {
-				var node = typeof elem.getAttributeNode !== "undefined" && elem.getAttributeNode("id");
-				return node && node.value === attrId;
-			};
-		};
-	}
-
-	// Tag
-	Expr.find["TAG"] = support.getElementsByTagName ?
-		function( tag, context ) {
-			if ( typeof context.getElementsByTagName !== "undefined" ) {
-				return context.getElementsByTagName( tag );
-
-			// DocumentFragment nodes don't have gEBTN
-			} else if ( support.qsa ) {
-				return context.querySelectorAll( tag );
-			}
-		} :
-
-		function( tag, context ) {
-			var elem,
-				tmp = [],
-				i = 0,
-				// By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
-				results = context.getElementsByTagName( tag );
-
-			// Filter out possible comments
-			if ( tag === "*" ) {
-				while ( (elem = results[i++]) ) {
-					if ( elem.nodeType === 1 ) {
-						tmp.push( elem );
-					}
-				}
-
-				return tmp;
-			}
-			return results;
-		};
-
-	// Class
-	Expr.find["CLASS"] = support.getElementsByClassName && function( className, context ) {
-		if ( documentIsHTML ) {
-			return context.getElementsByClassName( className );
-		}
-	};
-
-	/* QSA/matchesSelector
-	---------------------------------------------------------------------- */
-
-	// QSA and matchesSelector support
-
-	// matchesSelector(:active) reports false when true (IE9/Opera 11.5)
-	rbuggyMatches = [];
-
-	// qSa(:focus) reports false when true (Chrome 21)
-	// We allow this because of a bug in IE8/9 that throws an error
-	// whenever `document.activeElement` is accessed on an iframe
-	// So, we allow :focus to pass through QSA all the time to avoid the IE error
-	// See http://bugs.jquery.com/ticket/13378
-	rbuggyQSA = [];
-
-	if ( (support.qsa = rnative.test( doc.querySelectorAll )) ) {
-		// Build QSA regex
-		// Regex strategy adopted from Diego Perini
-		assert(function( div ) {
-			// Select is set to empty string on purpose
-			// This is to test IE's treatment of not explicitly
-			// setting a boolean content attribute,
-			// since its presence should be enough
-			// http://bugs.jquery.com/ticket/12359
-			docElem.appendChild( div ).innerHTML = "<a id='" + expando + "'></a>" +
-				"<select id='" + expando + "-\f]' msallowcapture=''>" +
-				"<option selected=''></option></select>";
-
-			// Support: IE8, Opera 11-12.16
-			// Nothing should be selected when empty strings follow ^= or $= or *=
-			// The test attribute must be unknown in Opera but "safe" for WinRT
-			// http://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
-			if ( div.querySelectorAll("[msallowcapture^='']").length ) {
-				rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
-			}
-
-			// Support: IE8
-			// Boolean attributes and "value" are not treated correctly
-			if ( !div.querySelectorAll("[selected]").length ) {
-				rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
-			}
-
-			// Support: Chrome<29, Android<4.2+, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.7+
-			if ( !div.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
-				rbuggyQSA.push("~=");
-			}
-
-			// Webkit/Opera - :checked should return selected option elements
-			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
-			// IE8 throws error here and will not see later tests
-			if ( !div.querySelectorAll(":checked").length ) {
-				rbuggyQSA.push(":checked");
-			}
-
-			// Support: Safari 8+, iOS 8+
-			// https://bugs.webkit.org/show_bug.cgi?id=136851
-			// In-page `selector#id sibing-combinator selector` fails
-			if ( !div.querySelectorAll( "a#" + expando + "+*" ).length ) {
-				rbuggyQSA.push(".#.+[+~]");
-			}
-		});
-
-		assert(function( div ) {
-			// Support: Windows 8 Native Apps
-			// The type and name attributes are restricted during .innerHTML assignment
-			var input = doc.createElement("input");
-			input.setAttribute( "type", "hidden" );
-			div.appendChild( input ).setAttribute( "name", "D" );
-
-			// Support: IE8
-			// Enforce case-sensitivity of name attribute
-			if ( div.querySelectorAll("[name=d]").length ) {
-				rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
-			}
-
-			// FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
-			// IE8 throws error here and will not see later tests
-			if ( !div.querySelectorAll(":enabled").length ) {
-				rbuggyQSA.push( ":enabled", ":disabled" );
-			}
-
-			// Opera 10-11 does not throw on post-comma invalid pseudos
-			div.querySelectorAll("*,:x");
-			rbuggyQSA.push(",.*:");
-		});
-	}
-
-	if ( (support.matchesSelector = rnative.test( (matches = docElem.matches ||
-		docElem.webkitMatchesSelector ||
-		docElem.mozMatchesSelector ||
-		docElem.oMatchesSelector ||
-		docElem.msMatchesSelector) )) ) {
-
-		assert(function( div ) {
-			// Check to see if it's possible to do matchesSelector
-			// on a disconnected node (IE 9)
-			support.disconnectedMatch = matches.call( div, "div" );
-
-			// This should fail with an exception
-			// Gecko does not error, returns false instead
-			matches.call( div, "[s!='']:x" );
-			rbuggyMatches.push( "!=", pseudos );
-		});
-	}
-
-	rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join("|") );
-	rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join("|") );
-
-	/* Contains
-	---------------------------------------------------------------------- */
-	hasCompare = rnative.test( docElem.compareDocumentPosition );
-
-	// Element contains another
-	// Purposefully does not implement inclusive descendent
-	// As in, an element does not contain itself
-	contains = hasCompare || rnative.test( docElem.contains ) ?
-		function( a, b ) {
-			var adown = a.nodeType === 9 ? a.documentElement : a,
-				bup = b && b.parentNode;
-			return a === bup || !!( bup && bup.nodeType === 1 && (
-				adown.contains ?
-					adown.contains( bup ) :
-					a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
-			));
-		} :
-		function( a, b ) {
-			if ( b ) {
-				while ( (b = b.parentNode) ) {
-					if ( b === a ) {
-						return true;
-					}
-				}
-			}
-			return false;
-		};
-
-	/* Sorting
-	---------------------------------------------------------------------- */
-
-	// Document order sorting
-	sortOrder = hasCompare ?
-	function( a, b ) {
-
-		// Flag for duplicate removal
-		if ( a === b ) {
-			hasDuplicate = true;
-			return 0;
-		}
-
-		// Sort on method existence if only one input has compareDocumentPosition
-		var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
-		if ( compare ) {
-			return compare;
-		}
-
-		// Calculate position if both inputs belong to the same document
-		compare = ( a.ownerDocument || a ) === ( b.ownerDocument || b ) ?
-			a.compareDocumentPosition( b ) :
-
-			// Otherwise we know they are disconnected
-			1;
-
-		// Disconnected nodes
-		if ( compare & 1 ||
-			(!support.sortDetached && b.compareDocumentPosition( a ) === compare) ) {
-
-			// Choose the first element that is related to our preferred document
-			if ( a === doc || a.ownerDocument === preferredDoc && contains(preferredDoc, a) ) {
-				return -1;
-			}
-			if ( b === doc || b.ownerDocument === preferredDoc && contains(preferredDoc, b) ) {
-				return 1;
-			}
-
-			// Maintain original order
-			return sortInput ?
-				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
-				0;
-		}
-
-		return compare & 4 ? -1 : 1;
-	} :
-	function( a, b ) {
-		// Exit early if the nodes are identical
-		if ( a === b ) {
-			hasDuplicate = true;
-			return 0;
-		}
-
-		var cur,
-			i = 0,
-			aup = a.parentNode,
-			bup = b.parentNode,
-			ap = [ a ],
-			bp = [ b ];
-
-		// Parentless nodes are either documents or disconnected
-		if ( !aup || !bup ) {
-			return a === doc ? -1 :
-				b === doc ? 1 :
-				aup ? -1 :
-				bup ? 1 :
-				sortInput ?
-				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
-				0;
-
-		// If the nodes are siblings, we can do a quick check
-		} else if ( aup === bup ) {
-			return siblingCheck( a, b );
-		}
-
-		// Otherwise we need full lists of their ancestors for comparison
-		cur = a;
-		while ( (cur = cur.parentNode) ) {
-			ap.unshift( cur );
-		}
-		cur = b;
-		while ( (cur = cur.parentNode) ) {
-			bp.unshift( cur );
-		}
-
-		// Walk down the tree looking for a discrepancy
-		while ( ap[i] === bp[i] ) {
-			i++;
-		}
-
-		return i ?
-			// Do a sibling check if the nodes have a common ancestor
-			siblingCheck( ap[i], bp[i] ) :
-
-			// Otherwise nodes in our document sort first
-			ap[i] === preferredDoc ? -1 :
-			bp[i] === preferredDoc ? 1 :
-			0;
-	};
-
-	return doc;
-};
-
-Sizzle.matches = function( expr, elements ) {
-	return Sizzle( expr, null, null, elements );
-};
-
-Sizzle.matchesSelector = function( elem, expr ) {
-	// Set document vars if needed
-	if ( ( elem.ownerDocument || elem ) !== document ) {
-		setDocument( elem );
-	}
-
-	// Make sure that attribute selectors are quoted
-	expr = expr.replace( rattributeQuotes, "='$1']" );
-
-	if ( support.matchesSelector && documentIsHTML &&
-		( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
-		( !rbuggyQSA     || !rbuggyQSA.test( expr ) ) ) {
-
-		try {
-			var ret = matches.call( elem, expr );
-
-			// IE 9's matchesSelector returns false on disconnected nodes
-			if ( ret || support.disconnectedMatch ||
-					// As well, disconnected nodes are said to be in a document
-					// fragment in IE 9
-					elem.document && elem.document.nodeType !== 11 ) {
-				return ret;
-			}
-		} catch (e) {}
-	}
-
-	return Sizzle( expr, document, null, [ elem ] ).length > 0;
-};
-
-Sizzle.contains = function( context, elem ) {
-	// Set document vars if needed
-	if ( ( context.ownerDocument || context ) !== document ) {
-		setDocument( context );
-	}
-	return contains( context, elem );
-};
-
-Sizzle.attr = function( elem, name ) {
-	// Set document vars if needed
-	if ( ( elem.ownerDocument || elem ) !== document ) {
-		setDocument( elem );
-	}
-
-	var fn = Expr.attrHandle[ name.toLowerCase() ],
-		// Don't get fooled by Object.prototype properties (jQuery #13807)
-		val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
-			fn( elem, name, !documentIsHTML ) :
-			undefined;
-
-	return val !== undefined ?
-		val :
-		support.attributes || !documentIsHTML ?
-			elem.getAttribute( name ) :
-			(val = elem.getAttributeNode(name)) && val.specified ?
-				val.value :
-				null;
-};
-
-Sizzle.error = function( msg ) {
-	throw new Error( "Syntax error, unrecognized expression: " + msg );
-};
-
-/**
- * Document sorting and removing duplicates
- * @param {ArrayLike} results
- */
-Sizzle.uniqueSort = function( results ) {
-	var elem,
-		duplicates = [],
-		j = 0,
-		i = 0;
-
-	// Unless we *know* we can detect duplicates, assume their presence
-	hasDuplicate = !support.detectDuplicates;
-	sortInput = !support.sortStable && results.slice( 0 );
-	results.sort( sortOrder );
-
-	if ( hasDuplicate ) {
-		while ( (elem = results[i++]) ) {
-			if ( elem === results[ i ] ) {
-				j = duplicates.push( i );
-			}
-		}
-		while ( j-- ) {
-			results.splice( duplicates[ j ], 1 );
-		}
-	}
-
-	// Clear input after sorting to release objects
-	// See https://github.com/jquery/sizzle/pull/225
-	sortInput = null;
-
-	return results;
-};
-
-/**
- * Utility function for retrieving the text value of an array of DOM nodes
- * @param {Array|Element} elem
- */
-getText = Sizzle.getText = function( elem ) {
-	var node,
-		ret = "",
-		i = 0,
-		nodeType = elem.nodeType;
-
-	if ( !nodeType ) {
-		// If no nodeType, this is expected to be an array
-		while ( (node = elem[i++]) ) {
-			// Do not traverse comment nodes
-			ret += getText( node );
-		}
-	} else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
-		// Use textContent for elements
-		// innerText usage removed for consistency of new lines (jQuery #11153)
-		if ( typeof elem.textContent === "string" ) {
-			return elem.textContent;
-		} else {
-			// Traverse its children
-			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
-				ret += getText( elem );
-			}
-		}
-	} else if ( nodeType === 3 || nodeType === 4 ) {
-		return elem.nodeValue;
-	}
-	// Do not include comment or processing instruction nodes
-
-	return ret;
-};
-
-Expr = Sizzle.selectors = {
-
-	// Can be adjusted by the user
-	cacheLength: 50,
-
-	createPseudo: markFunction,
-
-	match: matchExpr,
-
-	attrHandle: {},
-
-	find: {},
-
-	relative: {
-		">": { dir: "parentNode", first: true },
-		" ": { dir: "parentNode" },
-		"+": { dir: "previousSibling", first: true },
-		"~": { dir: "previousSibling" }
-	},
-
-	preFilter: {
-		"ATTR": function( match ) {
-			match[1] = match[1].replace( runescape, funescape );
-
-			// Move the given value to match[3] whether quoted or unquoted
-			match[3] = ( match[3] || match[4] || match[5] || "" ).replace( runescape, funescape );
-
-			if ( match[2] === "~=" ) {
-				match[3] = " " + match[3] + " ";
-			}
-
-			return match.slice( 0, 4 );
-		},
-
-		"CHILD": function( match ) {
-			/* matches from matchExpr["CHILD"]
-				1 type (only|nth|...)
-				2 what (child|of-type)
-				3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
-				4 xn-component of xn+y argument ([+-]?\d*n|)
-				5 sign of xn-component
-				6 x of xn-component
-				7 sign of y-component
-				8 y of y-component
-			*/
-			match[1] = match[1].toLowerCase();
-
-			if ( match[1].slice( 0, 3 ) === "nth" ) {
-				// nth-* requires argument
-				if ( !match[3] ) {
-					Sizzle.error( match[0] );
-				}
-
-				// numeric x and y parameters for Expr.filter.CHILD
-				// remember that false/true cast respectively to 0/1
-				match[4] = +( match[4] ? match[5] + (match[6] || 1) : 2 * ( match[3] === "even" || match[3] === "odd" ) );
-				match[5] = +( ( match[7] + match[8] ) || match[3] === "odd" );
-
-			// other types prohibit arguments
-			} else if ( match[3] ) {
-				Sizzle.error( match[0] );
-			}
-
-			return match;
-		},
-
-		"PSEUDO": function( match ) {
-			var excess,
-				unquoted = !match[6] && match[2];
-
-			if ( matchExpr["CHILD"].test( match[0] ) ) {
-				return null;
-			}
-
-			// Accept quoted arguments as-is
-			if ( match[3] ) {
-				match[2] = match[4] || match[5] || "";
-
-			// Strip excess characters from unquoted arguments
-			} else if ( unquoted && rpseudo.test( unquoted ) &&
-				// Get excess from tokenize (recursively)
-				(excess = tokenize( unquoted, true )) &&
-				// advance to the next closing parenthesis
-				(excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length) ) {
-
-				// excess is a negative index
-				match[0] = match[0].slice( 0, excess );
-				match[2] = unquoted.slice( 0, excess );
-			}
-
-			// Return only captures needed by the pseudo filter method (type and argument)
-			return match.slice( 0, 3 );
-		}
-	},
-
-	filter: {
-
-		"TAG": function( nodeNameSelector ) {
-			var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
-			return nodeNameSelector === "*" ?
-				function() { return true; } :
-				function( elem ) {
-					return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
-				};
-		},
-
-		"CLASS": function( className ) {
-			var pattern = classCache[ className + " " ];
-
-			return pattern ||
-				(pattern = new RegExp( "(^|" + whitespace + ")" + className + "(" + whitespace + "|$)" )) &&
-				classCache( className, function( elem ) {
-					return pattern.test( typeof elem.className === "string" && elem.className || typeof elem.getAttribute !== "undefined" && elem.getAttribute("class") || "" );
-				});
-		},
-
-		"ATTR": function( name, operator, check ) {
-			return function( elem ) {
-				var result = Sizzle.attr( elem, name );
-
-				if ( result == null ) {
-					return operator === "!=";
-				}
-				if ( !operator ) {
-					return true;
-				}
-
-				result += "";
-
-				return operator === "=" ? result === check :
-					operator === "!=" ? result !== check :
-					operator === "^=" ? check && result.indexOf( check ) === 0 :
-					operator === "*=" ? check && result.indexOf( check ) > -1 :
-					operator === "$=" ? check && result.slice( -check.length ) === check :
-					operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
-					operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
-					false;
-			};
-		},
-
-		"CHILD": function( type, what, argument, first, last ) {
-			var simple = type.slice( 0, 3 ) !== "nth",
-				forward = type.slice( -4 ) !== "last",
-				ofType = what === "of-type";
-
-			return first === 1 && last === 0 ?
-
-				// Shortcut for :nth-*(n)
-				function( elem ) {
-					return !!elem.parentNode;
-				} :
-
-				function( elem, context, xml ) {
-					var cache, outerCache, node, diff, nodeIndex, start,
-						dir = simple !== forward ? "nextSibling" : "previousSibling",
-						parent = elem.parentNode,
-						name = ofType && elem.nodeName.toLowerCase(),
-						useCache = !xml && !ofType;
-
-					if ( parent ) {
-
-						// :(first|last|only)-(child|of-type)
-						if ( simple ) {
-							while ( dir ) {
-								node = elem;
-								while ( (node = node[ dir ]) ) {
-									if ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) {
-										return false;
-									}
-								}
-								// Reverse direction for :only-* (if we haven't yet done so)
-								start = dir = type === "only" && !start && "nextSibling";
-							}
-							return true;
-						}
-
-						start = [ forward ? parent.firstChild : parent.lastChild ];
-
-						// non-xml :nth-child(...) stores cache data on `parent`
-						if ( forward && useCache ) {
-							// Seek `elem` from a previously-cached index
-							outerCache = parent[ expando ] || (parent[ expando ] = {});
-							cache = outerCache[ type ] || [];
-							nodeIndex = cache[0] === dirruns && cache[1];
-							diff = cache[0] === dirruns && cache[2];
-							node = nodeIndex && parent.childNodes[ nodeIndex ];
-
-							while ( (node = ++nodeIndex && node && node[ dir ] ||
-
-								// Fallback to seeking `elem` from the start
-								(diff = nodeIndex = 0) || start.pop()) ) {
-
-								// When found, cache indexes on `parent` and break
-								if ( node.nodeType === 1 && ++diff && node === elem ) {
-									outerCache[ type ] = [ dirruns, nodeIndex, diff ];
-									break;
-								}
-							}
-
-						// Use previously-cached element index if available
-						} else if ( useCache && (cache = (elem[ expando ] || (elem[ expando ] = {}))[ type ]) && cache[0] === dirruns ) {
-							diff = cache[1];
-
-						// xml :nth-child(...) or :nth-last-child(...) or :nth(-last)?-of-type(...)
-						} else {
-							// Use the same loop as above to seek `elem` from the start
-							while ( (node = ++nodeIndex && node && node[ dir ] ||
-								(diff = nodeIndex = 0) || start.pop()) ) {
-
-								if ( ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) && ++diff ) {
-									// Cache the index of each encountered element
-									if ( useCache ) {
-										(node[ expando ] || (node[ expando ] = {}))[ type ] = [ dirruns, diff ];
-									}
-
-									if ( node === elem ) {
-										break;
-									}
-								}
-							}
-						}
-
-						// Incorporate the offset, then check against cycle size
-						diff -= last;
-						return diff === first || ( diff % first === 0 && diff / first >= 0 );
-					}
-				};
-		},
-
-		"PSEUDO": function( pseudo, argument ) {
-			// pseudo-class names are case-insensitive
-			// http://www.w3.org/TR/selectors/#pseudo-classes
-			// Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
-			// Remember that setFilters inherits from pseudos
-			var args,
-				fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
-					Sizzle.error( "unsupported pseudo: " + pseudo );
-
-			// The user may use createPseudo to indicate that
-			// arguments are needed to create the filter function
-			// just as Sizzle does
-			if ( fn[ expando ] ) {
-				return fn( argument );
-			}
-
-			// But maintain support for old signatures
-			if ( fn.length > 1 ) {
-				args = [ pseudo, pseudo, "", argument ];
-				return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
-					markFunction(function( seed, matches ) {
-						var idx,
-							matched = fn( seed, argument ),
-							i = matched.length;
-						while ( i-- ) {
-							idx = indexOf( seed, matched[i] );
-							seed[ idx ] = !( matches[ idx ] = matched[i] );
-						}
-					}) :
-					function( elem ) {
-						return fn( elem, 0, args );
-					};
-			}
-
-			return fn;
-		}
-	},
-
-	pseudos: {
-		// Potentially complex pseudos
-		"not": markFunction(function( selector ) {
-			// Trim the selector passed to compile
-			// to avoid treating leading and trailing
-			// spaces as combinators
-			var input = [],
-				results = [],
-				matcher = compile( selector.replace( rtrim, "$1" ) );
-
-			return matcher[ expando ] ?
-				markFunction(function( seed, matches, context, xml ) {
-					var elem,
-						unmatched = matcher( seed, null, xml, [] ),
-						i = seed.length;
-
-					// Match elements unmatched by `matcher`
-					while ( i-- ) {
-						if ( (elem = unmatched[i]) ) {
-							seed[i] = !(matches[i] = elem);
-						}
-					}
-				}) :
-				function( elem, context, xml ) {
-					input[0] = elem;
-					matcher( input, null, xml, results );
-					// Don't keep the element (issue #299)
-					input[0] = null;
-					return !results.pop();
-				};
-		}),
-
-		"has": markFunction(function( selector ) {
-			return function( elem ) {
-				return Sizzle( selector, elem ).length > 0;
-			};
-		}),
-
-		"contains": markFunction(function( text ) {
-			text = text.replace( runescape, funescape );
-			return function( elem ) {
-				return ( elem.textContent || elem.innerText || getText( elem ) ).indexOf( text ) > -1;
-			};
-		}),
-
-		// "Whether an element is represented by a :lang() selector
-		// is based solely on the element's language value
-		// being equal to the identifier C,
-		// or beginning with the identifier C immediately followed by "-".
-		// The matching of C against the element's language value is performed case-insensitively.
-		// The identifier C does not have to be a valid language name."
-		// http://www.w3.org/TR/selectors/#lang-pseudo
-		"lang": markFunction( function( lang ) {
-			// lang value must be a valid identifier
-			if ( !ridentifier.test(lang || "") ) {
-				Sizzle.error( "unsupported lang: " + lang );
-			}
-			lang = lang.replace( runescape, funescape ).toLowerCase();
-			return function( elem ) {
-				var elemLang;
-				do {
-					if ( (elemLang = documentIsHTML ?
-						elem.lang :
-						elem.getAttribute("xml:lang") || elem.getAttribute("lang")) ) {
-
-						elemLang = elemLang.toLowerCase();
-						return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
-					}
-				} while ( (elem = elem.parentNode) && elem.nodeType === 1 );
-				return false;
-			};
-		}),
-
-		// Miscellaneous
-		"target": function( elem ) {
-			var hash = window.location && window.location.hash;
-			return hash && hash.slice( 1 ) === elem.id;
-		},
-
-		"root": function( elem ) {
-			return elem === docElem;
-		},
-
-		"focus": function( elem ) {
-			return elem === document.activeElement && (!document.hasFocus || document.hasFocus()) && !!(elem.type || elem.href || ~elem.tabIndex);
-		},
-
-		// Boolean properties
-		"enabled": function( elem ) {
-			return elem.disabled === false;
-		},
-
-		"disabled": function( elem ) {
-			return elem.disabled === true;
-		},
-
-		"checked": function( elem ) {
-			// In CSS3, :checked should return both checked and selected elements
-			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
-			var nodeName = elem.nodeName.toLowerCase();
-			return (nodeName === "input" && !!elem.checked) || (nodeName === "option" && !!elem.selected);
-		},
-
-		"selected": function( elem ) {
-			// Accessing this property makes selected-by-default
-			// options in Safari work properly
-			if ( elem.parentNode ) {
-				elem.parentNode.selectedIndex;
-			}
-
-			return elem.selected === true;
-		},
-
-		// Contents
-		"empty": function( elem ) {
-			// http://www.w3.org/TR/selectors/#empty-pseudo
-			// :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
-			//   but not by others (comment: 8; processing instruction: 7; etc.)
-			// nodeType < 6 works because attributes (2) do not appear as children
-			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
-				if ( elem.nodeType < 6 ) {
-					return false;
-				}
-			}
-			return true;
-		},
-
-		"parent": function( elem ) {
-			return !Expr.pseudos["empty"]( elem );
-		},
-
-		// Element/input types
-		"header": function( elem ) {
-			return rheader.test( elem.nodeName );
-		},
-
-		"input": function( elem ) {
-			return rinputs.test( elem.nodeName );
-		},
-
-		"button": function( elem ) {
-			var name = elem.nodeName.toLowerCase();
-			return name === "input" && elem.type === "button" || name === "button";
-		},
-
-		"text": function( elem ) {
-			var attr;
-			return elem.nodeName.toLowerCase() === "input" &&
-				elem.type === "text" &&
-
-				// Support: IE<8
-				// New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
-				( (attr = elem.getAttribute("type")) == null || attr.toLowerCase() === "text" );
-		},
-
-		// Position-in-collection
-		"first": createPositionalPseudo(function() {
-			return [ 0 ];
-		}),
-
-		"last": createPositionalPseudo(function( matchIndexes, length ) {
-			return [ length - 1 ];
-		}),
-
-		"eq": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			return [ argument < 0 ? argument + length : argument ];
-		}),
-
-		"even": createPositionalPseudo(function( matchIndexes, length ) {
-			var i = 0;
-			for ( ; i < length; i += 2 ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"odd": createPositionalPseudo(function( matchIndexes, length ) {
-			var i = 1;
-			for ( ; i < length; i += 2 ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"lt": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			var i = argument < 0 ? argument + length : argument;
-			for ( ; --i >= 0; ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"gt": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			var i = argument < 0 ? argument + length : argument;
-			for ( ; ++i < length; ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		})
-	}
-};
-
-Expr.pseudos["nth"] = Expr.pseudos["eq"];
-
-// Add button/input type pseudos
-for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
-	Expr.pseudos[ i ] = createInputPseudo( i );
-}
-for ( i in { submit: true, reset: true } ) {
-	Expr.pseudos[ i ] = createButtonPseudo( i );
-}
-
-// Easy API for creating new setFilters
-function setFilters() {}
-setFilters.prototype = Expr.filters = Expr.pseudos;
-Expr.setFilters = new setFilters();
-
-tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
-	var matched, match, tokens, type,
-		soFar, groups, preFilters,
-		cached = tokenCache[ selector + " " ];
-
-	if ( cached ) {
-		return parseOnly ? 0 : cached.slice( 0 );
-	}
-
-	soFar = selector;
-	groups = [];
-	preFilters = Expr.preFilter;
-
-	while ( soFar ) {
-
-		// Comma and first run
-		if ( !matched || (match = rcomma.exec( soFar )) ) {
-			if ( match ) {
-				// Don't consume trailing commas as valid
-				soFar = soFar.slice( match[0].length ) || soFar;
-			}
-			groups.push( (tokens = []) );
-		}
-
-		matched = false;
-
-		// Combinators
-		if ( (match = rcombinators.exec( soFar )) ) {
-			matched = match.shift();
-			tokens.push({
-				value: matched,
-				// Cast descendant combinators to space
-				type: match[0].replace( rtrim, " " )
-			});
-			soFar = soFar.slice( matched.length );
-		}
-
-		// Filters
-		for ( type in Expr.filter ) {
-			if ( (match = matchExpr[ type ].exec( soFar )) && (!preFilters[ type ] ||
-				(match = preFilters[ type ]( match ))) ) {
-				matched = match.shift();
-				tokens.push({
-					value: matched,
-					type: type,
-					matches: match
-				});
-				soFar = soFar.slice( matched.length );
-			}
-		}
-
-		if ( !matched ) {
-			break;
-		}
-	}
-
-	// Return the length of the invalid excess
-	// if we're just parsing
-	// Otherwise, throw an error or return tokens
-	return parseOnly ?
-		soFar.length :
-		soFar ?
-			Sizzle.error( selector ) :
-			// Cache the tokens
-			tokenCache( selector, groups ).slice( 0 );
-};
-
-function toSelector( tokens ) {
-	var i = 0,
-		len = tokens.length,
-		selector = "";
-	for ( ; i < len; i++ ) {
-		selector += tokens[i].value;
-	}
-	return selector;
-}
-
-function addCombinator( matcher, combinator, base ) {
-	var dir = combinator.dir,
-		checkNonElements = base && dir === "parentNode",
-		doneName = done++;
-
-	return combinator.first ?
-		// Check against closest ancestor/preceding element
-		function( elem, context, xml ) {
-			while ( (elem = elem[ dir ]) ) {
-				if ( elem.nodeType === 1 || checkNonElements ) {
-					return matcher( elem, context, xml );
-				}
-			}
-		} :
-
-		// Check against all ancestor/preceding elements
-		function( elem, context, xml ) {
-			var oldCache, outerCache,
-				newCache = [ dirruns, doneName ];
-
-			// We can't set arbitrary data on XML nodes, so they don't benefit from dir caching
-			if ( xml ) {
-				while ( (elem = elem[ dir ]) ) {
-					if ( elem.nodeType === 1 || checkNonElements ) {
-						if ( matcher( elem, context, xml ) ) {
-							return true;
-						}
-					}
-				}
-			} else {
-				while ( (elem = elem[ dir ]) ) {
-					if ( elem.nodeType === 1 || checkNonElements ) {
-						outerCache = elem[ expando ] || (elem[ expando ] = {});
-						if ( (oldCache = outerCache[ dir ]) &&
-							oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
-
-							// Assign to newCache so results back-propagate to previous elements
-							return (newCache[ 2 ] = oldCache[ 2 ]);
-						} else {
-							// Reuse newcache so results back-propagate to previous elements
-							outerCache[ dir ] = newCache;
-
-							// A match means we're done; a fail means we have to keep checking
-							if ( (newCache[ 2 ] = matcher( elem, context, xml )) ) {
-								return true;
-							}
-						}
-					}
-				}
-			}
-		};
-}
-
-function elementMatcher( matchers ) {
-	return matchers.length > 1 ?
-		function( elem, context, xml ) {
-			var i = matchers.length;
-			while ( i-- ) {
-				if ( !matchers[i]( elem, context, xml ) ) {
-					return false;
-				}
-			}
-			return true;
-		} :
-		matchers[0];
-}
-
-function multipleContexts( selector, contexts, results ) {
-	var i = 0,
-		len = contexts.length;
-	for ( ; i < len; i++ ) {
-		Sizzle( selector, contexts[i], results );
-	}
-	return results;
-}
-
-function condense( unmatched, map, filter, context, xml ) {
-	var elem,
-		newUnmatched = [],
-		i = 0,
-		len = unmatched.length,
-		mapped = map != null;
-
-	for ( ; i < len; i++ ) {
-		if ( (elem = unmatched[i]) ) {
-			if ( !filter || filter( elem, context, xml ) ) {
-				newUnmatched.push( elem );
-				if ( mapped ) {
-					map.push( i );
-				}
-			}
-		}
-	}
-
-	return newUnmatched;
-}
-
-function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
-	if ( postFilter && !postFilter[ expando ] ) {
-		postFilter = setMatcher( postFilter );
-	}
-	if ( postFinder && !postFinder[ expando ] ) {
-		postFinder = setMatcher( postFinder, postSelector );
-	}
-	return markFunction(function( seed, results, context, xml ) {
-		var temp, i, elem,
-			preMap = [],
-			postMap = [],
-			preexisting = results.length,
-
-			// Get initial elements from seed or context
-			elems = seed || multipleContexts( selector || "*", context.nodeType ? [ context ] : context, [] ),
-
-			// Prefilter to get matcher input, preserving a map for seed-results synchronization
-			matcherIn = preFilter && ( seed || !selector ) ?
-				condense( elems, preMap, preFilter, context, xml ) :
-				elems,
-
-			matcherOut = matcher ?
-				// If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
-				postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
-
-					// ...intermediate processing is necessary
-					[] :
-
-					// ...otherwise use results directly
-					results :
-				matcherIn;
-
-		// Find primary matches
-		if ( matcher ) {
-			matcher( matcherIn, matcherOut, context, xml );
-		}
-
-		// Apply postFilter
-		if ( postFilter ) {
-			temp = condense( matcherOut, postMap );
-			postFilter( temp, [], context, xml );
-
-			// Un-match failing elements by moving them back to matcherIn
-			i = temp.length;
-			while ( i-- ) {
-				if ( (elem = temp[i]) ) {
-					matcherOut[ postMap[i] ] = !(matcherIn[ postMap[i] ] = elem);
-				}
-			}
-		}
-
-		if ( seed ) {
-			if ( postFinder || preFilter ) {
-				if ( postFinder ) {
-					// Get the final matcherOut by condensing this intermediate into postFinder contexts
-					temp = [];
-					i = matcherOut.length;
-					while ( i-- ) {
-						if ( (elem = matcherOut[i]) ) {
-							// Restore matcherIn since elem is not yet a final match
-							temp.push( (matcherIn[i] = elem) );
-						}
-					}
-					postFinder( null, (matcherOut = []), temp, xml );
-				}
-
-				// Move matched elements from seed to results to keep them synchronized
-				i = matcherOut.length;
-				while ( i-- ) {
-					if ( (elem = matcherOut[i]) &&
-						(temp = postFinder ? indexOf( seed, elem ) : preMap[i]) > -1 ) {
-
-						seed[temp] = !(results[temp] = elem);
-					}
-				}
-			}
-
-		// Add elements to results, through postFinder if defined
-		} else {
-			matcherOut = condense(
-				matcherOut === results ?
-					matcherOut.splice( preexisting, matcherOut.length ) :
-					matcherOut
-			);
-			if ( postFinder ) {
-				postFinder( null, results, matcherOut, xml );
-			} else {
-				push.apply( results, matcherOut );
-			}
-		}
-	});
-}
-
-function matcherFromTokens( tokens ) {
-	var checkContext, matcher, j,
-		len = tokens.length,
-		leadingRelative = Expr.relative[ tokens[0].type ],
-		implicitRelative = leadingRelative || Expr.relative[" "],
-		i = leadingRelative ? 1 : 0,
-
-		// The foundational matcher ensures that elements are reachable from top-level context(s)
-		matchContext = addCombinator( function( elem ) {
-			return elem === checkContext;
-		}, implicitRelative, true ),
-		matchAnyContext = addCombinator( function( elem ) {
-			return indexOf( checkContext, elem ) > -1;
-		}, implicitRelative, true ),
-		matchers = [ function( elem, context, xml ) {
-			var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
-				(checkContext = context).nodeType ?
-					matchContext( elem, context, xml ) :
-					matchAnyContext( elem, context, xml ) );
-			// Avoid hanging onto element (issue #299)
-			checkContext = null;
-			return ret;
-		} ];
-
-	for ( ; i < len; i++ ) {
-		if ( (matcher = Expr.relative[ tokens[i].type ]) ) {
-			matchers = [ addCombinator(elementMatcher( matchers ), matcher) ];
-		} else {
-			matcher = Expr.filter[ tokens[i].type ].apply( null, tokens[i].matches );
-
-			// Return special upon seeing a positional matcher
-			if ( matcher[ expando ] ) {
-				// Find the next relative operator (if any) for proper handling
-				j = ++i;
-				for ( ; j < len; j++ ) {
-					if ( Expr.relative[ tokens[j].type ] ) {
-						break;
-					}
-				}
-				return setMatcher(
-					i > 1 && elementMatcher( matchers ),
-					i > 1 && toSelector(
-						// If the preceding token was a descendant combinator, insert an implicit any-element `*`
-						tokens.slice( 0, i - 1 ).concat({ value: tokens[ i - 2 ].type === " " ? "*" : "" })
-					).replace( rtrim, "$1" ),
-					matcher,
-					i < j && matcherFromTokens( tokens.slice( i, j ) ),
-					j < len && matcherFromTokens( (tokens = tokens.slice( j )) ),
-					j < len && toSelector( tokens )
-				);
-			}
-			matchers.push( matcher );
-		}
-	}
-
-	return elementMatcher( matchers );
-}
-
-function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
-	var bySet = setMatchers.length > 0,
-		byElement = elementMatchers.length > 0,
-		superMatcher = function( seed, context, xml, results, outermost ) {
-			var elem, j, matcher,
-				matchedCount = 0,
-				i = "0",
-				unmatched = seed && [],
-				setMatched = [],
-				contextBackup = outermostContext,
-				// We must always have either seed elements or outermost context
-				elems = seed || byElement && Expr.find["TAG"]( "*", outermost ),
-				// Use integer dirruns iff this is the outermost matcher
-				dirrunsUnique = (dirruns += contextBackup == null ? 1 : Math.random() || 0.1),
-				len = elems.length;
-
-			if ( outermost ) {
-				outermostContext = context !== document && context;
-			}
-
-			// Add elements passing elementMatchers directly to results
-			// Keep `i` a string if there are no elements so `matchedCount` will be "00" below
-			// Support: IE<9, Safari
-			// Tolerate NodeList properties (IE: "length"; Safari: <number>) matching elements by id
-			for ( ; i !== len && (elem = elems[i]) != null; i++ ) {
-				if ( byElement && elem ) {
-					j = 0;
-					while ( (matcher = elementMatchers[j++]) ) {
-						if ( matcher( elem, context, xml ) ) {
-							results.push( elem );
-							break;
-						}
-					}
-					if ( outermost ) {
-						dirruns = dirrunsUnique;
-					}
-				}
-
-				// Track unmatched elements for set filters
-				if ( bySet ) {
-					// They will have gone through all possible matchers
-					if ( (elem = !matcher && elem) ) {
-						matchedCount--;
-					}
-
-					// Lengthen the array for every element, matched or not
-					if ( seed ) {
-						unmatched.push( elem );
-					}
-				}
-			}
-
-			// Apply set filters to unmatched elements
-			matchedCount += i;
-			if ( bySet && i !== matchedCount ) {
-				j = 0;
-				while ( (matcher = setMatchers[j++]) ) {
-					matcher( unmatched, setMatched, context, xml );
-				}
-
-				if ( seed ) {
-					// Reintegrate element matches to eliminate the need for sorting
-					if ( matchedCount > 0 ) {
-						while ( i-- ) {
-							if ( !(unmatched[i] || setMatched[i]) ) {
-								setMatched[i] = pop.call( results );
-							}
-						}
-					}
-
-					// Discard index placeholder values to get only actual matches
-					setMatched = condense( setMatched );
-				}
-
-				// Add matches to results
-				push.apply( results, setMatched );
-
-				// Seedless set matches succeeding multiple successful matchers stipulate sorting
-				if ( outermost && !seed && setMatched.length > 0 &&
-					( matchedCount + setMatchers.length ) > 1 ) {
-
-					Sizzle.uniqueSort( results );
-				}
-			}
-
-			// Override manipulation of globals by nested matchers
-			if ( outermost ) {
-				dirruns = dirrunsUnique;
-				outermostContext = contextBackup;
-			}
-
-			return unmatched;
-		};
-
-	return bySet ?
-		markFunction( superMatcher ) :
-		superMatcher;
-}
-
-compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
-	var i,
-		setMatchers = [],
-		elementMatchers = [],
-		cached = compilerCache[ selector + " " ];
-
-	if ( !cached ) {
-		// Generate a function of recursive functions that can be used to check each element
-		if ( !match ) {
-			match = tokenize( selector );
-		}
-		i = match.length;
-		while ( i-- ) {
-			cached = matcherFromTokens( match[i] );
-			if ( cached[ expando ] ) {
-				setMatchers.push( cached );
-			} else {
-				elementMatchers.push( cached );
-			}
-		}
-
-		// Cache the compiled function
-		cached = compilerCache( selector, matcherFromGroupMatchers( elementMatchers, setMatchers ) );
-
-		// Save selector and tokenization
-		cached.selector = selector;
-	}
-	return cached;
-};
-
-/**
- * A low-level selection function that works with Sizzle's compiled
- *  selector functions
- * @param {String|Function} selector A selector or a pre-compiled
- *  selector function built with Sizzle.compile
- * @param {Element} context
- * @param {Array} [results]
- * @param {Array} [seed] A set of elements to match against
- */
-select = Sizzle.select = function( selector, context, results, seed ) {
-	var i, tokens, token, type, find,
-		compiled = typeof selector === "function" && selector,
-		match = !seed && tokenize( (selector = compiled.selector || selector) );
-
-	results = results || [];
-
-	// Try to minimize operations if there is no seed and only one group
-	if ( match.length === 1 ) {
-
-		// Take a shortcut and set the context if the root selector is an ID
-		tokens = match[0] = match[0].slice( 0 );
-		if ( tokens.length > 2 && (token = tokens[0]).type === "ID" &&
-				support.getById && context.nodeType === 9 && documentIsHTML &&
-				Expr.relative[ tokens[1].type ] ) {
-
-			context = ( Expr.find["ID"]( token.matches[0].replace(runescape, funescape), context ) || [] )[0];
-			if ( !context ) {
-				return results;
-
-			// Precompiled matchers will still verify ancestry, so step up a level
-			} else if ( compiled ) {
-				context = context.parentNode;
-			}
-
-			selector = selector.slice( tokens.shift().value.length );
-		}
-
-		// Fetch a seed set for right-to-left matching
-		i = matchExpr["needsContext"].test( selector ) ? 0 : tokens.length;
-		while ( i-- ) {
-			token = tokens[i];
-
-			// Abort if we hit a combinator
-			if ( Expr.relative[ (type = token.type) ] ) {
-				break;
-			}
-			if ( (find = Expr.find[ type ]) ) {
-				// Search, expanding context for leading sibling combinators
-				if ( (seed = find(
-					token.matches[0].replace( runescape, funescape ),
-					rsibling.test( tokens[0].type ) && testContext( context.parentNode ) || context
-				)) ) {
-
-					// If seed is empty or no tokens remain, we can return early
-					tokens.splice( i, 1 );
-					selector = seed.length && toSelector( tokens );
-					if ( !selector ) {
-						push.apply( results, seed );
-						return results;
-					}
-
-					break;
-				}
-			}
-		}
-	}
-
-	// Compile and execute a filtering function if one is not provided
-	// Provide `match` to avoid retokenization if we modified the selector above
-	( compiled || compile( selector, match ) )(
-		seed,
-		context,
-		!documentIsHTML,
-		results,
-		rsibling.test( selector ) && testContext( context.parentNode ) || context
-	);
-	return results;
-};
-
-// One-time assignments
-
-// Sort stability
-support.sortStable = expando.split("").sort( sortOrder ).join("") === expando;
-
-// Support: Chrome 14-35+
-// Always assume duplicates if they aren't passed to the comparison function
-support.detectDuplicates = !!hasDuplicate;
-
-// Initialize against the default document
-setDocument();
-
-// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27)
-// Detached nodes confoundingly follow *each other*
-support.sortDetached = assert(function( div1 ) {
-	// Should return 1, but returns 4 (following)
-	return div1.compareDocumentPosition( document.createElement("div") ) & 1;
-});
-
-// Support: IE<8
-// Prevent attribute/property "interpolation"
-// http://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
-if ( !assert(function( div ) {
-	div.innerHTML = "<a href='#'></a>";
-	return div.firstChild.getAttribute("href") === "#" ;
-}) ) {
-	addHandle( "type|href|height|width", function( elem, name, isXML ) {
-		if ( !isXML ) {
-			return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
-		}
-	});
-}
-
-// Support: IE<9
-// Use defaultValue in place of getAttribute("value")
-if ( !support.attributes || !assert(function( div ) {
-	div.innerHTML = "<input/>";
-	div.firstChild.setAttribute( "value", "" );
-	return div.firstChild.getAttribute( "value" ) === "";
-}) ) {
-	addHandle( "value", function( elem, name, isXML ) {
-		if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
-			return elem.defaultValue;
-		}
-	});
-}
-
-// Support: IE<9
-// Use getAttributeNode to fetch booleans when getAttribute lies
-if ( !assert(function( div ) {
-	return div.getAttribute("disabled") == null;
-}) ) {
-	addHandle( booleans, function( elem, name, isXML ) {
-		var val;
-		if ( !isXML ) {
-			return elem[ name ] === true ? name.toLowerCase() :
-					(val = elem.getAttributeNode( name )) && val.specified ?
-					val.value :
-				null;
-		}
-	});
-}
-
-return Sizzle;
-
-})( window );
-
-
-
-jQuery.find = Sizzle;
-jQuery.expr = Sizzle.selectors;
-jQuery.expr[":"] = jQuery.expr.pseudos;
-jQuery.unique = Sizzle.uniqueSort;
-jQuery.text = Sizzle.getText;
-jQuery.isXMLDoc = Sizzle.isXML;
-jQuery.contains = Sizzle.contains;
-
-
-
-var rneedsContext = jQuery.expr.match.needsContext;
-
-var rsingleTag = (/^<(\w+)\s*\/?>(?:<\/\1>|)$/);
-
-
-
-var risSimple = /^.[^:#\[\.,]*$/;
-
-// Implement the identical functionality for filter and not
-function winnow( elements, qualifier, not ) {
-	if ( jQuery.isFunction( qualifier ) ) {
-		return jQuery.grep( elements, function( elem, i ) {
-			/* jshint -W018 */
-			return !!qualifier.call( elem, i, elem ) !== not;
-		});
-
-	}
-
-	if ( qualifier.nodeType ) {
-		return jQuery.grep( elements, function( elem ) {
-			return ( elem === qualifier ) !== not;
-		});
-
-	}
-
-	if ( typeof qualifier === "string" ) {
-		if ( risSimple.test( qualifier ) ) {
-			return jQuery.filter( qualifier, elements, not );
-		}
-
-		qualifier = jQuery.filter( qualifier, elements );
-	}
-
-	return jQuery.grep( elements, function( elem ) {
-		return ( indexOf.call( qualifier, elem ) >= 0 ) !== not;
-	});
-}
-
-jQuery.filter = function( expr, elems, not ) {
-	var elem = elems[ 0 ];
-
-	if ( not ) {
-		expr = ":not(" + expr + ")";
-	}
-
-	return elems.length === 1 && elem.nodeType === 1 ?
-		jQuery.find.matchesSelector( elem, expr ) ? [ elem ] : [] :
-		jQuery.find.matches( expr, jQuery.grep( elems, function( elem ) {
-			return elem.nodeType === 1;
-		}));
-};
-
-jQuery.fn.extend({
-	find: function( selector ) {
-		var i,
-			len = this.length,
-			ret = [],
-			self = this;
-
-		if ( typeof selector !== "string" ) {
-			return this.pushStack( jQuery( selector ).filter(function() {
-				for ( i = 0; i < len; i++ ) {
-					if ( jQuery.contains( self[ i ], this ) ) {
-						return true;
-					}
-				}
-			}) );
-		}
-
-		for ( i = 0; i < len; i++ ) {
-			jQuery.find( selector, self[ i ], ret );
-		}
-
-		// Needed because $( selector, context ) becomes $( context ).find( selector )
-		ret = this.pushStack( len > 1 ? jQuery.unique( ret ) : ret );
-		ret.selector = this.selector ? this.selector + " " + selector : selector;
-		return ret;
-	},
-	filter: function( selector ) {
-		return this.pushStack( winnow(this, selector || [], false) );
-	},
-	not: function( selector ) {
-		return this.pushStack( winnow(this, selector || [], true) );
-	},
-	is: function( selector ) {
-		return !!winnow(
-			this,
-
-			// If this is a positional/relative selector, check membership in the returned set
-			// so $("p:first").is("p:last") won't return true for a doc with two "p".
-			typeof selector === "string" && rneedsContext.test( selector ) ?
-				jQuery( selector ) :
-				selector || [],
-			false
-		).length;
-	}
-});
-
-
-// Initialize a jQuery object
-
-
-// A central reference to the root jQuery(document)
-var rootjQuery,
-
-	// A simple way to check for HTML strings
-	// Prioritize #id over <tag> to avoid XSS via location.hash (#9521)
-	// Strict HTML recognition (#11290: must start with <)
-	rquickExpr = /^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,
-
-	init = jQuery.fn.init = function( selector, context ) {
-		var match, elem;
-
-		// HANDLE: $(""), $(null), $(undefined), $(false)
-		if ( !selector ) {
-			return this;
-		}
-
-		// Handle HTML strings
-		if ( typeof selector === "string" ) {
-			if ( selector[0] === "<" && selector[ selector.length - 1 ] === ">" && selector.length >= 3 ) {
-				// Assume that strings that start and end with <> are HTML and skip the regex check
-				match = [ null, selector, null ];
-
-			} else {
-				match = rquickExpr.exec( selector );
-			}
-
-			// Match html or make sure no context is specified for #id
-			if ( match && (match[1] || !context) ) {
-
-				// HANDLE: $(html) -> $(array)
-				if ( match[1] ) {
-					context = context instanceof jQuery ? context[0] : context;
-
-					// Option to run scripts is true for back-compat
-					// Intentionally let the error be thrown if parseHTML is not present
-					jQuery.merge( this, jQuery.parseHTML(
-						match[1],
-						context && context.nodeType ? context.ownerDocument || context : document,
-						true
-					) );
-
-					// HANDLE: $(html, props)
-					if ( rsingleTag.test( match[1] ) && jQuery.isPlainObject( context ) ) {
-						for ( match in context ) {
-							// Properties of context are called as methods if possible
-							if ( jQuery.isFunction( this[ match ] ) ) {
-								this[ match ]( context[ match ] );
-
-							// ...and otherwise set as attributes
-							} else {
-								this.attr( match, context[ match ] );
-							}
-						}
-					}
-
-					return this;
-
-				// HANDLE: $(#id)
-				} else {
-					elem = document.getElementById( match[2] );
-
-					// Support: Blackberry 4.6
-					// gEBID returns nodes no longer in the document (#6963)
-					if ( elem && elem.parentNode ) {
-						// Inject the element directly into the jQuery object
-						this.length = 1;
-						this[0] = elem;
-					}
-
-					this.context = document;
-					this.selector = selector;
-					return this;
-				}
-
-			// HANDLE: $(expr, $(...))
-			} else if ( !context || context.jquery ) {
-				return ( context || rootjQuery ).find( selector );
-
-			// HANDLE: $(expr, context)
-			// (which is just equivalent to: $(context).find(expr)
-			} else {
-				return this.constructor( context ).find( selector );
-			}
-
-		// HANDLE: $(DOMElement)
-		} else if ( selector.nodeType ) {
-			this.context = this[0] = selector;
-			this.length = 1;
-			return this;
-
-		// HANDLE: $(function)
-		// Shortcut for document ready
-		} else if ( jQuery.isFunction( selector ) ) {
-			return typeof rootjQuery.ready !== "undefined" ?
-				rootjQuery.ready( selector ) :
-				// Execute immediately if ready is not present
-				selector( jQuery );
-		}
-
-		if ( selector.selector !== undefined ) {
-			this.selector = selector.selector;
-			this.context = selector.context;
-		}
-
-		return jQuery.makeArray( selector, this );
-	};
-
-// Give the init function the jQuery prototype for later instantiation
-init.prototype = jQuery.fn;
-
-// Initialize central reference
-rootjQuery = jQuery( document );
-
-
-var rparentsprev = /^(?:parents|prev(?:Until|All))/,
-	// Methods guaranteed to produce a unique set when starting from a unique set
-	guaranteedUnique = {
-		children: true,
-		contents: true,
-		next: true,
-		prev: true
-	};
-
-jQuery.extend({
-	dir: function( elem, dir, until ) {
-		var matched = [],
-			truncate = until !== undefined;
-
-		while ( (elem = elem[ dir ]) && elem.nodeType !== 9 ) {
-			if ( elem.nodeType === 1 ) {
-				if ( truncate && jQuery( elem ).is( until ) ) {
-					break;
-				}
-				matched.push( elem );
-			}
-		}
-		return matched;
-	},
-
-	sibling: function( n, elem ) {
-		var matched = [];
-
-		for ( ; n; n = n.nextSibling ) {
-			if ( n.nodeType === 1 && n !== elem ) {
-				matched.push( n );
-			}
-		}
-
-		return matched;
-	}
-});
-
-jQuery.fn.extend({
-	has: function( target ) {
-		var targets = jQuery( target, this ),
-			l = targets.length;
-
-		return this.filter(function() {
-			var i = 0;
-			for ( ; i < l; i++ ) {
-				if ( jQuery.contains( this, targets[i] ) ) {
-					return true;
-				}
-			}
-		});
-	},
-
-	closest: function( selectors, context ) {
-		var cur,
-			i = 0,
-			l = this.length,
-			matched = [],
-			pos = rneedsContext.test( selectors ) || typeof selectors !== "string" ?
-				jQuery( selectors, context || this.context ) :
-				0;
-
-		for ( ; i < l; i++ ) {
-			for ( cur = this[i]; cur && cur !== context; cur = cur.parentNode ) {
-				// Always skip document fragments
-				if ( cur.nodeType < 11 && (pos ?
-					pos.index(cur) > -1 :
-
-					// Don't pass non-elements to Sizzle
-					cur.nodeType === 1 &&
-						jQuery.find.matchesSelector(cur, selectors)) ) {
-
-					matched.push( cur );
-					break;
-				}
-			}
-		}
-
-		return this.pushStack( matched.length > 1 ? jQuery.unique( matched ) : matched );
-	},
-
-	// Determine the position of an element within the set
-	index: function( elem ) {
-
-		// No argument, return index in parent
-		if ( !elem ) {
-			return ( this[ 0 ] && this[ 0 ].parentNode ) ? this.first().prevAll().length : -1;
-		}
-
-		// Index in selector
-		if ( typeof elem === "string" ) {
-			return indexOf.call( jQuery( elem ), this[ 0 ] );
-		}
-
-		// Locate the position of the desired element
-		return indexOf.call( this,
-
-			// If it receives a jQuery object, the first element is used
-			elem.jquery ? elem[ 0 ] : elem
-		);
-	},
-
-	add: function( selector, context ) {
-		return this.pushStack(
-			jQuery.unique(
-				jQuery.merge( this.get(), jQuery( selector, context ) )
-			)
-		);
-	},
-
-	addBack: function( selector ) {
-		return this.add( selector == null ?
-			this.prevObject : this.prevObject.filter(selector)
-		);
-	}
-});
-
-function sibling( cur, dir ) {
-	while ( (cur = cur[dir]) && cur.nodeType !== 1 ) {}
-	return cur;
-}
-
-jQuery.each({
-	parent: function( elem ) {
-		var parent = elem.parentNode;
-		return parent && parent.nodeType !== 11 ? parent : null;
-	},
-	parents: function( elem ) {
-		return jQuery.dir( elem, "parentNode" );
-	},
-	parentsUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "parentNode", until );
-	},
-	next: function( elem ) {
-		return sibling( elem, "nextSibling" );
-	},
-	prev: function( elem ) {
-		return sibling( elem, "previousSibling" );
-	},
-	nextAll: function( elem ) {
-		return jQuery.dir( elem, "nextSibling" );
-	},
-	prevAll: function( elem ) {
-		return jQuery.dir( elem, "previousSibling" );
-	},
-	nextUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "nextSibling", until );
-	},
-	prevUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "previousSibling", until );
-	},
-	siblings: function( elem ) {
-		return jQuery.sibling( ( elem.parentNode || {} ).firstChild, elem );
-	},
-	children: function( elem ) {
-		return jQuery.sibling( elem.firstChild );
-	},
-	contents: function( elem ) {
-		return elem.contentDocument || jQuery.merge( [], elem.childNodes );
-	}
-}, function( name, fn ) {
-	jQuery.fn[ name ] = function( until, selector ) {
-		var matched = jQuery.map( this, fn, until );
-
-		if ( name.slice( -5 ) !== "Until" ) {
-			selector = until;
-		}
-
-		if ( selector && typeof selector === "string" ) {
-			matched = jQuery.filter( selector, matched );
-		}
-
-		if ( this.length > 1 ) {
-			// Remove duplicates
-			if ( !guaranteedUnique[ name ] ) {
-				jQuery.unique( matched );
-			}
-
-			// Reverse order for parents* and prev-derivatives
-			if ( rparentsprev.test( name ) ) {
-				matched.reverse();
-			}
-		}
-
-		return this.pushStack( matched );
-	};
-});
-var rnotwhite = (/\S+/g);
-
-
-
-// String to Object options format cache
-var optionsCache = {};
-
-// Convert String-formatted options into Object-formatted ones and store in cache
-function createOptions( options ) {
-	var object = optionsCache[ options ] = {};
-	jQuery.each( options.match( rnotwhite ) || [], function( _, flag ) {
-		object[ flag ] = true;
-	});
-	return object;
-}
-
-/*
- * Create a callback list using the following parameters:
- *
- *	options: an optional list of space-separated options that will change how
- *			the callback list behaves or a more traditional option object
- *
- * By default a callback list will act like an event callback list and can be
- * "fired" multiple times.
- *
- * Possible options:
- *
- *	once:			will ensure the callback list can only be fired once (like a Deferred)
- *
- *	memory:			will keep track of previous values and will call any callback added
- *					after the list has been fired right away with the latest "memorized"
- *					values (like a Deferred)
- *
- *	unique:			will ensure a callback can only be added once (no duplicate in the list)
- *
- *	stopOnFalse:	interrupt callings when a callback returns false
- *
- */
-jQuery.Callbacks = function( options ) {
-
-	// Convert options from String-formatted to Object-formatted if needed
-	// (we check in cache first)
-	options = typeof options === "string" ?
-		( optionsCache[ options ] || createOptions( options ) ) :
-		jQuery.extend( {}, options );
-
-	var // Last fire value (for non-forgettable lists)
-		memory,
-		// Flag to know if list was already fired
-		fired,
-		// Flag to know if list is currently firing
-		firing,
-		// First callback to fire (used internally by add and fireWith)
-		firingStart,
-		// End of the loop when firing
-		firingLength,
-		// Index of currently firing callback (modified by remove if needed)
-		firingIndex,
-		// Actual callback list
-		list = [],
-		// Stack of fire calls for repeatable lists
-		stack = !options.once && [],
-		// Fire callbacks
-		fire = function( data ) {
-			memory = options.memory && data;
-			fired = true;
-			firingIndex = firingStart || 0;
-			firingStart = 0;
-			firingLength = list.length;
-			firing = true;
-			for ( ; list && firingIndex < firingLength; firingIndex++ ) {
-				if ( list[ firingIndex ].apply( data[ 0 ], data[ 1 ] ) === false && options.stopOnFalse ) {
-					memory = false; // To prevent further calls using add
-					break;
-				}
-			}
-			firing = false;
-			if ( list ) {
-				if ( stack ) {
-					if ( stack.length ) {
-						fire( stack.shift() );
-					}
-				} else if ( memory ) {
-					list = [];
-				} else {
-					self.disable();
-				}
-			}
-		},
-		// Actual Callbacks object
-		self = {
-			// Add a callback or a collection of callbacks to the list
-			add: function() {
-				if ( list ) {
-					// First, we save the current length
-					var start = list.length;
-					(function add( args ) {
-						jQuery.each( args, function( _, arg ) {
-							var type = jQuery.type( arg );
-							if ( type === "function" ) {
-								if ( !options.unique || !self.has( arg ) ) {
-									list.push( arg );
-								}
-							} else if ( arg && arg.length && type !== "string" ) {
-								// Inspect recursively
-								add( arg );
-							}
-						});
-					})( arguments );
-					// Do we need to add the callbacks to the
-					// current firing batch?
-					if ( firing ) {
-						firingLength = li

<TRUNCATED>

[11/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
deleted file mode 100644
index da6b75b..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
+++ /dev/null
@@ -1,33 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th>Key</th>
-      <th>Value</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="entry in jobmanager.config | orderBy: 'key'">
-      <td>{{entry.key}}</td>
-      <td>{{entry.value}}</td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
deleted file mode 100644
index 02c2f47..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
+++ /dev/null
@@ -1,33 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Job Manager</div>
-</nav>
-<nav class="navbar navbar-default navbar-fixed-top navbar-main-additional">
-  <ul class="nav nav-tabs">
-    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
-    <li ui-sref-active="active"><a ui-sref=".log">Logs</a></li>
-    <li ui-sref-active="active"><a ui-sref=".stdout">Stdout</a></li>
-  </ul>
-</nav>
-<div id="content-inner" class="has-navbar-main-additional">
-  <div ui-view="details"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
deleted file mode 100644
index df6a817..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">
-        <div class="row">
-          <div class="col-xs-10">Job Manager Output</div>
-          <div class="col-xs-1 text-right"><a ng-click="reloadData()" class="show-pointer"><i class="fa fa-refresh"></i></a></div>
-          <div class="col-xs-1 text-left"><a href="jobmanager/stdout"><i class="fa fa-download"></i></a></div>
-        </div>
-      </th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td colspan="2">
-        <pre>{{jobmanager.stdout}}</pre>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
deleted file mode 100644
index b76278d..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
+++ /dev/null
@@ -1,53 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Completed Jobs</div>
-</nav>
-<div id="content-inner">
-  <table class="table table-hover table-clickable">
-    <thead>
-      <tr>
-        <th>Start Time</th>
-        <th>End Time</th>
-        <th>Duration</th>
-        <th>Job Name</th>
-        <th>Job ID</th>
-        <th>Tasks</th>
-        <th>Status</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job.duration}} ms</td>
-        <td>{{job.name}}</td>
-        <td>{{job.jid}}</td>
-        <td class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-        </td>
-        <td> 
-          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-        </td>
-      </tr>
-    </tbody>
-  </table>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
deleted file mode 100644
index a7a5d9d..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
+++ /dev/null
@@ -1,57 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table ng-if="job['execution-config']" class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">Execution configuration</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Execution mode</td>
-      <td>{{ job['execution-config']['execution-mode'] }}</td>
-    </tr>
-    <tr>
-      <td>Max. number of execution retries</td>
-      <td>{{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }}</td>
-    </tr>
-    <tr>
-      <td>Job parallelism</td>
-      <td>{{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }}</td>
-    </tr>
-    <tr>
-      <td>Object reuse mode</td>
-      <td>{{ job['execution-config']['object-reuse-mode'] }}</td>
-    </tr>
-  </tbody>
-</table>
-<table ng-if="job['execution-config']['user-config']" class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">User configuration</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="property in job['execution-config']['user-config']">
-      <td>{{property.name}}</td>
-      <td table-property="table-property" value="property.value"></td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/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
deleted file mode 100644
index a5f6676..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
+++ /dev/null
@@ -1,38 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="exceptions['root-exception']" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">Root exception</div>
-  </div>
-  <div class="panel-body">
-    <pre class="exception">{{ exceptions['root-exception'] }}</pre>
-  </div>
-</div>
-<div ng-repeat="exception in exceptions['all-exceptions']" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <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>
-  <div class="panel-body">
-    <pre class="exception">{{ exception.exception }}</pre>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
deleted file mode 100644
index 9d3e171..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
+++ /dev/null
@@ -1,48 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">
-    <indicator-primary status="{{job.state}}"></indicator-primary>{{ job.name }}
-  </div>
-  <div class="navbar-info first last hidden-xs hidden-sm">{{ job.jid }}</div>
-  <div class="navbar-info first last">
-    <div class="label-group">
-      <bs-label status="{{status}}" ng-repeat="(status, value) in job['status-counts']">{{value}}</bs-label>
-    </div>
-  </div>
-  <div class="navbar-info first last hidden-xs hidden-sm">{{ job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}<span ng-if="job['end-time'] &gt; -1">
-      - 
-      {{ job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></div>
-  <div ng-if="job.duration &gt; -1" class="navbar-info last first">{{job.duration}} ms</div>
-</nav>
-<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
-  <ul class="nav nav-tabs">
-    <li ui-sref-active="active"><a ui-sref=".plan.overview">Plan</a></li>
-    <li ui-sref-active="active"><a ui-sref=".statistics">Job Accumulators / Statistics</a></li>
-    <li ui-sref-active="active"><a ui-sref=".timeline">Timeline</a></li>
-    <li ui-sref-active="active"><a ui-sref=".exceptions">Exceptions</a></li>
-    <li ui-sref-active="active"><a ui-sref=".properties">Properties</a></li>
-    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
-  </ul>
-</nav>
-<div id="content-inner" class="has-navbar-main-additional">
-  <div ui-view="details"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
deleted file mode 100644
index f2c4143..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
+++ /dev/null
@@ -1,31 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
-</div>
-<div ng-if="plan" class="panel panel-default panel-multi">
-  <nav class="navbar navbar-default navbar-secondary-additional">
-    <ul class="nav nav-tabs">
-      <li ui-sref-active="active"><a ui-sref=".overview({nodeid: nodeid})">Overview</a></li>
-      <li ui-sref-active="active"><a ui-sref=".accumulators({nodeid: nodeid})">Accumulators</a></li>
-    </ul>
-  </nav>
-  <div ui-view="node-details" class="panel-body clean"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
deleted file mode 100644
index 8de3921..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-body-hover table-clickable table-activable">
-  <thead>
-    <tr>
-      <th>Name</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="v.id == nodeid || changeNode(v.id)">
-    <tr ng-if="v.type == 'regular'">
-      <td>{{ v.name | humanizeText }}</td>
-      <td> 
-        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
-      </td>
-    </tr>
-    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
-      <td colspan="10">
-        <div ng-include=" 'partials/jobs/job.plan.node.accumulators.html' "></div>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
deleted file mode 100644
index 1706d3e..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
+++ /dev/null
@@ -1,60 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-body-hover table-clickable table-activable">
-  <thead>
-    <tr>
-      <th>Start Time</th>
-      <th>End Time</th>
-      <th>Duration</th>
-      <th>Name</th>
-      <th>Bytes read</th>
-      <th>Records read</th>
-      <th>Bytes written</th>
-      <th>Records written</th>
-      <th>Tasks</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="changeNode(v.id)">
-    <tr ng-if="v.type == 'regular'">
-      <td><span ng-if="v['start-time'] &gt; -1">{{ v['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="v['end-time'] &gt; -1">{{ v['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="v.duration &gt; -1">{{ v.duration }} ms</span></td>
-      <td class="td-long">{{ v.name | humanizeText }}</td>
-      <td>{{ v.metrics['read-bytes'] }}</td>
-      <td>{{ v.metrics['read-records'] }}</td>
-      <td>{{ v.metrics['write-bytes'] }}</td>
-      <td>{{ v.metrics['write-records'] }}</td>
-      <td>
-        <div class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(index, status) in stateList">{{v.tasks[status]}}</bs-label>
-        </div>
-      </td>
-      <td> 
-        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
-      </td>
-    </tr>
-    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
-      <td colspan="10">
-        <div ng-include=" 'partials/jobs/job.plan.node.subtasks.html' "></div>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
deleted file mode 100644
index e7dcf2c..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
+++ /dev/null
@@ -1,68 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="accumulators.length == 0">
-  <p><i>No accumulators</i></p>
-</div>
-<div ng-if="accumulators &amp;&amp; accumulators.length &gt; 0">
-  <table class="table table-hover table-clickable table-activable table-inner">
-    <thead>
-      <tr>
-        <th>Name</th>
-        <th>Type</th>
-        <th>Value</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="accumulator in accumulators">
-        <td width="30%">{{ accumulator.name }}</td>
-        <td width="30%">{{ accumulator.type }}</td>
-        <td width="30%">{{ accumulator.value }}</td>
-      </tr>
-    </tbody>
-  </table>
-  <div ng-if="!nodeUnfolded"><a ng-click="toggleFold()" class="btn btn-default">
-      Show subtasks
-       <i class="fa fa-chevron-down"></i></a><a ng-click="deactivateNode(); $event.stopPropagation()" title="Fold" class="btn btn-default pull-right"><i class="fa fa-chevron-up"></i></a></div>
-  <div ng-if="nodeUnfolded &amp;&amp; subtaskAccumulators &amp;&amp; subtaskAccumulators.length &gt; 0"><a ng-click="toggleFold()" class="btn btn-default">
-      Hide subtasks
-       <i class="fa fa-chevron-up"></i></a>
-    <table class="table table-hover table-clickable table-activable table-inner">
-      <thead>
-        <tr>
-          <th>Name</th>
-          <th>Type</th>
-          <th>Value</th>
-        </tr>
-      </thead>
-      <tbody ng-if="subtask['user-accumulators'] &amp;&amp; subtask['user-accumulators'].length &gt; 0" ng-repeat="subtask in subtaskAccumulators">
-        <tr>
-          <td colwidth="3">
-            <div class="small-label">({{ subtask.subtask }}) {{ subtask.host }}, attempt: {{ subtask.attempt + 1 }}</div>
-          </td>
-        </tr>
-        <tr ng-repeat="accumulator in subtask['user-accumulators']">
-          <td width="30%">{{ accumulator.name }}</td>
-          <td width="30%">{{ accumulator.type }}</td>
-          <td width="30%">{{ accumulator.value }}</td>
-        </tr>
-      </tbody>
-    </table>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
deleted file mode 100644
index 40b16bc..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
+++ /dev/null
@@ -1,52 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-
--->
-<table ng-if="subtasks" class="table table-hover table-clickable table-activable table-inner">
-  <thead>
-    <tr>
-      <th>Start Time</th>
-      <th>End Time</th>
-      <th>Duration</th>
-      <th>Bytes read</th>
-      <th>Records read</th>
-      <th>Bytes written</th>
-      <th>Records written</th>
-      <th>Attempt</th>
-      <th>Host</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="subtask in subtasks">
-      <td><span ng-if="subtask['start-time'] &gt; -1">{{ subtask['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="subtask['end-time'] &gt; -1">{{ subtask['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="subtask.duration &gt; -1">{{ subtask.duration }} ms</span></td>
-      <td><span ng-if="subtask.metrics['read-bytes'] &gt; -1">{{ subtask.metrics['read-bytes'] }}</span></td>
-      <td><span ng-if="subtask.metrics['read-records'] &gt; -1">{{ subtask.metrics['read-records'] }}</span></td>
-      <td><span ng-if="subtask.metrics['write-bytes'] &gt; -1">{{ subtask.metrics['write-bytes'] }}</span></td>
-      <td><span ng-if="subtask.metrics['write-records'] &gt; -1">{{ subtask.metrics['write-records'] }}</span></td>
-      <td>{{ subtask.attempt + 1 }}</td>
-      <td>{{ subtask.host }}</td>
-      <td> 
-        <bs-label status="{{subtask.status}}">{{subtask.status}}</bs-label>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
deleted file mode 100644
index 907afd3..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
+++ /dev/null
@@ -1,140 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
-</div>
-<div ng-if="node" class="panel panel-default">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">{{ node.description | humanizeText }}</div>
-  </div>
-  <div class="panel-body clean">
-    <div class="row">
-      <div class="col-sm-6 col-md-4">
-        <table ng-if="node.optimizer_properties.global_properties" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Global Data Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.global_properties">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-        <table ng-if="node.optimizer_properties.local_properties" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Local Data Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.local_properties">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-        <div class="visible-xs visible-sm">
-          <table class="table table-properties">
-            <thead>
-              <tr>
-                <th colspan="2">Properties</th>
-              </tr>
-            </thead>
-            <tbody>
-              <tr>
-                <td>Operator</td>
-                <td table-property="table-property" value="node.operator_strategy"></td>
-              </tr>
-              <tr>
-                <td>Parallelism</td>
-                <td table-property="table-property" value="node.parallelism"></td>
-              </tr>
-            </tbody>
-          </table>
-        </div>
-      </div>
-      <div class="hidden-sm col-md-4">
-        <table class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr>
-              <td>Operator</td>
-              <td table-property="table-property" value="node.operator_strategy"></td>
-            </tr>
-            <tr>
-              <td>Parallelism</td>
-              <td table-property="table-property" value="node.parallelism"></td>
-            </tr>
-          </tbody>
-        </table>
-        <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Size Estimates</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.estimates">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-      </div>
-      <div class="col-sm-6 col-md-4">
-        <div class="visible-xs visible-sm">
-          <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
-            <thead>
-              <tr>
-                <th colspan="2">Size Estimates</th>
-              </tr>
-            </thead>
-            <tbody>
-              <tr ng-repeat="property in node.optimizer_properties.estimates">
-                <td>{{property.name}}</td>
-                <td table-property="table-property" value="property.value"></td>
-              </tr>
-            </tbody>
-          </table>
-        </div>
-        <table ng-if="node.optimizer_properties.costs" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Cost Estimates</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.costs">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-      </div>
-    </div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
deleted file mode 100644
index 951cc1c..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">Some statistics</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Operator</td>
-      <td>1</td>
-    </tr>
-    <tr>
-      <td>Parallelism</td>
-      <td>2</td>
-    </tr>
-    <tr>
-      <td>Subtasks-per-instance</td>
-      <td>3</td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
deleted file mode 100644
index 2f22576..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
+++ /dev/null
@@ -1,23 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div timeline="timeline" vertices="vertices" jobid="jobid" class="timeline-canvas"></div>
-</div>
-<div ui-view="vertex"></div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
deleted file mode 100644
index 1a4bd06..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
+++ /dev/null
@@ -1,30 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="vertex" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">{{ vertex.groupvertex.groupvertexname | humanizeText }}</div>
-  </div>
-  <div class="panel-body">
-    <div class="canvas-wrapper">
-      <div vertex="vertex" data="vertex" class="timeline-canvas"></div>
-    </div>
-    <div id="timeline1"></div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
deleted file mode 100644
index e175d07..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
+++ /dev/null
@@ -1,53 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Running Jobs</div>
-</nav>
-<div id="content-inner">
-  <table class="table table-hover table-clickable">
-    <thead>
-      <tr>
-        <th>Start Time</th>
-        <th>End Time</th>
-        <th>Duration</th>
-        <th>Job Name</th>
-        <th>Job ID</th>
-        <th>Tasks</th>
-        <th>Status</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job.duration}} ms</td>
-        <td>{{job.name}}</td>
-        <td>{{job.jid}}</td>
-        <td class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-        </td>
-        <td> 
-          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-        </td>
-      </tr>
-    </tbody>
-  </table>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/overview.html b/flink-runtime-web/web-dashboard/web/partials/overview.html
deleted file mode 100644
index ec3c580..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/overview.html
+++ /dev/null
@@ -1,147 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Overview</div>
-</nav>
-<div id="content-inner">
-  <div class="row">
-    <div class="col-md-6">
-      <div class="panel panel-default panel-dashboard">
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-tasks fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview.taskmanagers}}</div>
-              <div>Task Managers</div>
-            </div>
-          </div>
-        </div>
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-folder fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview["slots-total"]}}</div>
-              <div>Task Slots</div>
-            </div>
-          </div>
-        </div>
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-folder-o fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview["slots-available"]}}</div>
-              <div>Available Task Slots</div>
-            </div>
-          </div>
-        </div>
-      </div>
-    </div>
-    <div class="col-md-6">
-      <div class="panel panel-default panel-lg">
-        <div class="panel-heading">Total Jobs</div>
-        <div class="list-group">
-          <div class="list-group-item">
-            <div class="badge badge-primary">{{overview["jobs-running"]}}</div>Running
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-success">{{overview["jobs-finished"]}}</div>Finished
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-info">{{overview["jobs-cancelled"]}}</div>Canceled
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-danger">{{overview["jobs-failed"]}}</div>Failed
-          </div>
-        </div>
-      </div>
-    </div>
-  </div>
-  <div class="panel panel-default">
-    <div class="panel-heading">
-      <h3 class="panel-title">Running Jobs</h3>
-    </div>
-    <div class="panel-body">
-      <table class="table table-hover table-clickable">
-        <thead>
-          <tr>
-            <th>Start Time</th>
-            <th>End Time</th>
-            <th>Duration</th>
-            <th>Job Name</th>
-            <th>Job ID</th>
-            <th>Tasks</th>
-            <th>Status</th>
-          </tr>
-        </thead>
-        <tbody>
-          <tr ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job.duration}} ms</td>
-            <td>{{job.name}}</td>
-            <td>{{job.jid}}</td>
-            <td class="label-group">
-              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-            </td>
-            <td> 
-              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-            </td>
-          </tr>
-        </tbody>
-      </table>
-    </div>
-  </div>
-  <div class="panel panel-default">
-    <div class="panel-heading">
-      <h3 class="panel-title">Completed Jobs</h3>
-    </div>
-    <div class="panel-body">
-      <table class="table table-hover table-clickable">
-        <thead>
-          <tr>
-            <th>Start Time</th>
-            <th>End Time</th>
-            <th>Duration</th>
-            <th>Job Name</th>
-            <th>Job ID</th>
-            <th>Tasks</th>
-            <th>Status</th>
-          </tr>
-        </thead>
-        <tbody>
-          <tr ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job.duration}} ms</td>
-            <td>{{job.name}}</td>
-            <td>{{job.jid}}</td>
-            <td class="label-group">
-              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-            </td>
-            <td> 
-              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-            </td>
-          </tr>
-        </tbody>
-      </table>
-    </div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html b/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
deleted file mode 100644
index bf37409..0000000
--- a/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
+++ /dev/null
@@ -1,57 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Task Managers</div>
-</nav>
-<div id="content-inner">
-  <table class="table table-clickable table-hover">
-    <thead>
-      <tr>
-        <th>Path, ID</th>
-        <th>Data Port</th>
-        <th>Last Heartbeat</th>
-        <th>All Slots</th>
-        <th>Free Slots</th>
-        <th>CPU Cores</th>
-        <th>Physical Memory</th>
-        <th>Free Memory</th>
-        <th>Flink Managed Memory</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="manager in managers" ui-sref="single-manager.metrics({taskmanagerid: manager.id})">
-        <td>
-          {{ manager.path }}
-           
-          <div class="small-label">{{ manager.id }}</div>
-        </td>
-        <td>{{ manager.dataPort }}</td>
-        <td>{{ manager.timeSinceLastHeartbeat | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</td>
-        <td>{{ manager.slotsNumber }}</td>
-        <td>{{ manager.freeSlots }}</td>
-        <td>{{ manager.cpuCores }}</td>
-        <td>{{ manager.physicalMemory | bytes:MB }}</td>
-        <td>{{ manager.freeMemory | bytes:MB }}</td>
-        <td>{{ manager.managedMemory | bytes:MB }}</td>
-      </tr>
-    </tbody>
-  </table>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
index c00e7fe..a2b0183 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
@@ -156,4 +156,9 @@ public class AkkaActorGateway implements ActorGateway, Serializable {
 	public UUID leaderSessionID() {
 		return leaderSessionID;
 	}
+
+	@Override
+	public String toString() {
+		return String.format("AkkaActorGateway(%s, %s)", actor.path(), leaderSessionID);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
index 4cdc4f8..460f10e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
@@ -63,6 +63,8 @@ public final class StandaloneUtils {
 			Configuration configuration,
 			String jobManagerName)
 		throws UnknownHostException {
+
+
 		Tuple2<String, Object> stringIntPair = TaskManager.getAndCheckJobManagerAddress(configuration);
 
 		String jobManagerHostname = stringIntPair._1();

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitor.java
index c2e2b0d..63e2cc8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitor.java
@@ -28,7 +28,7 @@ public interface WebMonitor {
 	 * 
 	 * @throws Exception This method may forward exceptions, if it cannot bring up the web monitor.
 	 */
-	void start() throws Exception;
+	void start(String jobManagerAkkaUrl) throws Exception;
 
 	/**
 	 * Stops the web server.

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/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 055274d..161e8de 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
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.jobmanager
 
 import java.io.{File, IOException}
 import java.lang.reflect.{Constructor, InvocationTargetException}
-import java.net.InetSocketAddress
+import java.net.{UnknownHostException, InetAddress, InetSocketAddress}
 import java.util.UUID
 
 import akka.actor.Status.Failure
@@ -43,7 +43,7 @@ import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
+import org.apache.flink.runtime.leaderretrieval.{StandaloneLeaderRetrievalService, LeaderRetrievalService}
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -130,6 +130,16 @@ class JobManager(
   var futuresToComplete: Option[Seq[Future[Unit]]] = None
 
   /**
+   * The port of the web monitor as configured. Make sure that it is actually configured before
+   * starting the JobManager. This tightly couples the web monitor with the job manager. It is a
+   * temporary workaround until all execution graph components are properly serializable and all
+   * web monitors can transparently interact with each job manager. Currently each web server has
+   * to run in the actor system of the associated job manager.
+   */
+  val webMonitorPort : Int = flinkConfiguration.getInteger(
+    ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1)
+
+  /**
    * Run when the job manager is started. Simply logs an informational message.
    * The method also starts the leader election service.
    */
@@ -729,6 +739,9 @@ class JobManager(
 
     case RequestLeaderSessionID =>
       sender() ! ResponseLeaderSessionID(leaderSessionID.orNull)
+
+    case RequestWebMonitorPort =>
+      sender() ! ResponseWebMonitorPort(webMonitorPort)
   }
 
   /**
@@ -1529,6 +1542,39 @@ object JobManager {
       }
     }
 
+    val webMonitor: Option[WebMonitor] =
+      if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
+        val address = AkkaUtils.getAddress(jobManagerSystem)
+
+        configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
+        configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
+
+        // start the job manager web frontend
+        if (configuration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) {
+          val leaderRetrievalService = LeaderRetrievalUtils
+            .createLeaderRetrievalService(configuration)
+
+          LOG.info("Starting NEW JobManger web frontend")
+          // start the new web frontend. we need to load this dynamically
+          // because it is not in the same project/dependencies
+          Some(startWebRuntimeMonitor(configuration, leaderRetrievalService, jobManagerSystem))
+        }
+        else {
+          LOG.info("Starting JobManger web frontend")
+
+          // The old web frontend does not work with recovery mode
+          val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(configuration)
+          Some(new WebInfoServer(configuration, leaderRetrievalService, jobManagerSystem))
+        }
+      }
+      else {
+        None
+      }
+
+    // Reset the port (necessary in case of automatic port selection)
+    webMonitor.foreach{ monitor => configuration.setInteger(
+      ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, monitor.getServerPort) }
+
     try {
       // bring up the job manager actor
       LOG.info("Starting JobManager actor")
@@ -1574,19 +1620,10 @@ object JobManager {
           "TaskManager_Process_Reaper")
       }
 
-      val webMonitor = if (
-        configuration.getInteger(
-          ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
-          0) >= 0) {
-
-        // TODO: Add support for HA. Webserver has to work in dedicated mode. All transferred
-        // information has to be made serializable
-        val address = AkkaUtils.getAddress(jobManagerSystem)
-
-        configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
-        configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
-
-        val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(configuration)
+      webMonitor.foreach {
+        monitor =>
+          val jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(configuration)
+          monitor.start(jobManagerAkkaUrl)
 
         LOG.info("Starting JobManger web frontend")
         // start the web frontend. we need to load this dynamically
@@ -1595,11 +1632,6 @@ object JobManager {
           configuration,
           leaderRetrievalService,
           jobManagerSystem)
-
-        if(webServer != null) {
-          webServer.start()
-        }
-
         Option(webServer)
       } else {
         None
@@ -1978,6 +2010,31 @@ object JobManager {
   }
 
   /**
+   * Returns the JobManager actor's remote Akka URL, given the configured hostname and port.
+   *
+   * @param config The configuration to parse
+   * @return JobManager actor remote Akka URL
+   */
+  def getRemoteJobManagerAkkaURL(config: Configuration) : String = {
+    val (hostname, port) = TaskManager.getAndCheckJobManagerAddress(config)
+
+    var hostPort: InetSocketAddress = null
+
+    try {
+      val inetAddress: InetAddress = InetAddress.getByName(hostname)
+      hostPort = new InetSocketAddress(inetAddress, port)
+    }
+    catch {
+      case e: UnknownHostException => {
+        throw new UnknownHostException(s"Cannot resolve the JobManager hostname '$hostname' " +
+          s"specified in the configuration")
+      }
+    }
+
+    JobManager.getRemoteJobManagerAkkaURL(hostPort, Option.empty)
+  }
+
+  /**
    * Builds the akka actor path for the JobManager actor to address the actor within
    * its own actor system.
    *

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index 8a4d27b..bb5f598 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -380,6 +380,16 @@ object JobManagerMessages {
   /** Response containing the ActorRef of the archiver */
   case class ResponseArchive(actor: ActorRef)
 
+  /** Request for the [[org.apache.flink.runtime.webmonitor.WebMonitor]] port. */
+  case object RequestWebMonitorPort
+
+  /**
+   * Response containing the [[org.apache.flink.runtime.webmonitor.WebMonitor]] port.
+   *
+   * -1 indicates that there is no web monitor running.
+   */
+  case class ResponseWebMonitorPort(port: Integer)
+
   // --------------------------------------------------------------------------
   // Utility methods to allow simpler case object access from Java
   // --------------------------------------------------------------------------
@@ -431,4 +441,8 @@ object JobManagerMessages {
   def getRecoverAllJobs: AnyRef = {
     RecoverAllJobs
   }
+  
+  def getRequestWebMonitorPort: AnyRef = {
+    RequestWebMonitorPort
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index 9c4381e..cefb462 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -292,7 +292,7 @@ abstract class FlinkMiniCluster(
       val webServer = WebMonitorUtils.startWebRuntimeMonitor(
         config, leaderRetrievalService, actorSystem)
 
-      webServer.start()
+      webServer.start(jobManagerAkkaURL)
 
       Option(webServer)
     } else {

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
index ad162fe..6fcbcef 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
@@ -19,6 +19,7 @@ package org.apache.flink.yarn
 
 import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager}
 
+
 /** Default implemenation of the [[ApplicationMasterBase]] which starts a [[YarnJobManager]] and a
   * [[MemoryArchivist]].
   */

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index efe2101..47c47c5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -735,7 +735,7 @@ under the License.
 						<exclude>flink-runtime-web/web-dashboard/assets/fonts/fontawesome*</exclude>
 
                         <!-- generated contents -->
-						<exclude>flink-runtime-web/web-dashboard/web/**</exclude>
+						<exclude>flink-runtime-web/src/main/resources/web/**</exclude>
 
 						<!-- downloaded and generated web libraries. -->
 						<exclude>flink-runtime-web/web-dashboard/node_modules/**</exclude>


[32/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
deleted file mode 100644
index 0086ac6..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-
-import java.io.Serializable;
-
-/**
- * State handler provider factory.
- *
- * <p>This is going to be superseded soon.
- */
-public class StateHandleProviderFactory {
-
-	/**
-	 * Creates a {@link org.apache.flink.runtime.state.FileStateHandle.FileStateHandleProvider} at
-	 * the configured recovery path.
-	 */
-	public static <T extends Serializable> StateHandleProvider<T> createRecoveryFileStateHandleProvider(
-			Configuration config) {
-
-		StateBackend stateBackend = StateBackend.fromConfig(config);
-
-		if (stateBackend == StateBackend.FILESYSTEM) {
-			String recoveryPath = config.getString(
-					ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
-
-			if (recoveryPath.equals("")) {
-				throw new IllegalConfigurationException("Missing recovery path. Specify via " +
-						"configuration key '" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "'.");
-			}
-			else {
-				return FileStateHandle.createProvider(recoveryPath);
-			}
-		}
-		else {
-			throw new IllegalConfigurationException("Unexpected state backend configuration " +
-					stateBackend);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java
new file mode 100644
index 0000000..32c601e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamStateHandle.java
@@ -0,0 +1,28 @@
+/*
+ * 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.state;
+
+import org.apache.flink.runtime.state.StateHandle;
+
+import java.io.InputStream;
+
+/**
+ * A state handle that produces an input stream when resolved.
+ */
+public interface StreamStateHandle extends StateHandle<InputStream> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java
new file mode 100644
index 0000000..d64e2c4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileState.java
@@ -0,0 +1,83 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+
+import java.io.IOException;
+
+/**
+ * Base class for state that is stored in a file.
+ */
+public abstract class AbstractFileState implements java.io.Serializable {
+	
+	private static final long serialVersionUID = 350284443258002355L;
+	
+	/** The path to the file in the filesystem, fully describing the file system */
+	private final Path filePath;
+
+	/** Cached file system handle */
+	private transient FileSystem fs;
+
+	/**
+	 * Creates a new file state for the given file path.
+	 * 
+	 * @param filePath The path to the file that stores the state.
+	 */
+	protected AbstractFileState(Path filePath) {
+		this.filePath = filePath;
+	}
+
+	/**
+	 * Gets the path where this handle's state is stored.
+	 * @return The path where this handle's state is stored.
+	 */
+	public Path getFilePath() {
+		return filePath;
+	}
+
+	/**
+	 * Discard the state by deleting the file that stores the state. If the parent directory
+	 * of the state is empty after deleting the state file, it is also deleted.
+	 * 
+	 * @throws Exception Thrown, if the file deletion (not the directory deletion) fails.
+	 */
+	public void discardState() throws Exception {
+		getFileSystem().delete(filePath, false);
+
+		// send a call to delete the directory containing the file. this will
+		// fail (and be ignored) when some files still exist
+		try {
+			getFileSystem().delete(filePath.getParent(), false);
+		} catch (IOException ignored) {}
+	}
+
+	/**
+	 * Gets the file system that stores the file state.
+	 * @return The file system that stores the file state.
+	 * @throws IOException Thrown if the file system cannot be accessed.
+	 */
+	protected FileSystem getFileSystem() throws IOException {
+		if (fs == null) {
+			fs = FileSystem.get(filePath.toUri());
+		}
+		return fs;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java
new file mode 100644
index 0000000..b7e7cd1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java
@@ -0,0 +1,53 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.ObjectInputStream;
+
+/**
+ * A state handle that points to state stored in a file via Java Serialization.
+ * 
+ * @param <T> The type of state pointed to by the state handle.
+ */
+public class FileSerializableStateHandle<T> extends AbstractFileState implements StateHandle<T> {
+
+	private static final long serialVersionUID = -657631394290213622L;
+	
+	/**
+	 * Creates a new FileSerializableStateHandle pointing to state at the given file path.
+	 * 
+	 * @param filePath The path to the file containing the checkpointed state.
+	 */
+	public FileSerializableStateHandle(Path filePath) {
+		super(filePath);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public T getState(ClassLoader classLoader) throws Exception {
+		FSDataInputStream inStream = getFileSystem().open(getFilePath());
+		ObjectInputStream ois = new InstantiationUtil.ClassLoaderObjectInputStream(inStream, classLoader);
+		return (T) ois.readObject();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStreamStateHandle.java
new file mode 100644
index 0000000..f4681ea
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStreamStateHandle.java
@@ -0,0 +1,46 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import java.io.InputStream;
+
+/**
+ * A state handle that points to state in a file system, accessible as an input stream.
+ */
+public class FileStreamStateHandle extends AbstractFileState implements StreamStateHandle {
+	
+	private static final long serialVersionUID = -6826990484549987311L;
+
+	/**
+	 * Creates a new FileStreamStateHandle pointing to state at the given file path.
+	 * 
+	 * @param filePath The path to the file containing the checkpointed state.
+	 */
+	public FileStreamStateHandle(Path filePath) {
+		super(filePath);
+	}
+
+	@Override
+	public InputStream getState(ClassLoader userCodeClassLoader) throws Exception {
+		return getFileSystem().open(getFilePath());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvState.java
new file mode 100644
index 0000000..e3116dd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvState.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.filesystem;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.runtime.state.AbstractHeapKvState;
+
+import java.io.DataOutputStream;
+import java.util.HashMap;
+
+/**
+ * Heap-backed key/value state that is snapshotted into files.
+ * 
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public class FsHeapKvState<K, V> extends AbstractHeapKvState<K, V, FsStateBackend> {
+	
+	/** The file system state backend backing snapshots of this state */
+	private final FsStateBackend backend;
+	
+	/**
+	 * Creates a new and empty key/value state.
+	 * 
+	 * @param keySerializer The serializer for the key.
+	 * @param valueSerializer The serializer for the value.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+	 * @param backend The file system state backend backing snapshots of this state
+	 */
+	public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+							V defaultValue, FsStateBackend backend) {
+		super(keySerializer, valueSerializer, defaultValue);
+		this.backend = backend;
+	}
+
+	/**
+	 * Creates a new key/value state with the given state contents.
+	 * This method is used to re-create key/value state with existing data, for example from
+	 * a snapshot.
+	 * 
+	 * @param keySerializer The serializer for the key.
+	 * @param valueSerializer The serializer for the value.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+	 * @param state The map of key/value pairs to initialize the state with.
+	 * @param backend The file system state backend backing snapshots of this state
+	 */
+	public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+							V defaultValue, HashMap<K, V> state, FsStateBackend backend) {
+		super(keySerializer, valueSerializer, defaultValue, state);
+		this.backend = backend;
+	}
+
+	
+	@Override
+	public FsHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
+		// first, create an output stream to write to
+		try (FsStateBackend.FsCheckpointStateOutputStream out = 
+					backend.createCheckpointStateOutputStream(checkpointId, timestamp)) {
+
+			// serialize the state to the output stream
+			OutputViewDataOutputStreamWrapper outView = 
+					new OutputViewDataOutputStreamWrapper(new DataOutputStream(out));
+			outView.writeInt(size());
+			writeStateToOutputView(outView);
+			outView.flush();
+			
+			// create a handle to the state
+			return new FsHeapKvStateSnapshot<>(getKeySerializer(), getValueSerializer(), out.closeAndGetPath());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvStateSnapshot.java
new file mode 100644
index 0000000..781ee3d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsHeapKvStateSnapshot.java
@@ -0,0 +1,95 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.runtime.state.KvStateSnapshot;
+
+import java.io.DataInputStream;
+import java.util.HashMap;
+
+/**
+ * A snapshot of a heap key/value state stored in a file.
+ * 
+ * @param <K> The type of the key in the snapshot state.
+ * @param <V> The type of the value in the snapshot state.
+ */
+public class FsHeapKvStateSnapshot<K, V> extends AbstractFileState implements KvStateSnapshot<K, V, FsStateBackend> {
+	
+	private static final long serialVersionUID = 1L;
+
+	/** Name of the key serializer class */
+	private final String keySerializerClassName;
+
+	/** Name of the value serializer class */
+	private final String valueSerializerClassName;
+
+	/**
+	 * Creates a new state snapshot with data in the file system.
+	 *
+	 * @param keySerializer The serializer for the keys.
+	 * @param valueSerializer The serializer for the values.
+	 * @param filePath The path where the snapshot data is stored.
+	 */
+	public FsHeapKvStateSnapshot(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, Path filePath) {
+		super(filePath);
+		this.keySerializerClassName = keySerializer.getClass().getName();
+		this.valueSerializerClassName = valueSerializer.getClass().getName();
+	}
+
+	@Override
+	public FsHeapKvState<K, V> restoreState(
+			FsStateBackend stateBackend,
+			final TypeSerializer<K> keySerializer,
+			final TypeSerializer<V> valueSerializer,
+			V defaultValue,
+			ClassLoader classLoader) throws Exception {
+
+		// validity checks
+		if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
+				!valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
+			throw new IllegalArgumentException(
+					"Cannot restore the state from the snapshot with the given serializers. " +
+							"State (K/V) was serialized with (" + valueSerializerClassName +
+							"/" + keySerializerClassName + ")");
+		}
+		
+		// state restore
+		try (FSDataInputStream inStream = stateBackend.getFileSystem().open(getFilePath())) {
+			InputViewDataInputStreamWrapper inView = new InputViewDataInputStreamWrapper(new DataInputStream(inStream));
+			
+			final int numEntries = inView.readInt();
+			HashMap<K, V> stateMap = new HashMap<>(numEntries);
+			
+			for (int i = 0; i < numEntries; i++) {
+				K key = keySerializer.deserialize(inView);
+				V value = valueSerializer.deserialize(inView);
+				stateMap.put(key, value);
+			}
+			
+			return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap, stateBackend);
+		}
+		catch (Exception e) {
+			throw new Exception("Failed to restore state from file system", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
new file mode 100644
index 0000000..045c411
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
@@ -0,0 +1,412 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.UUID;
+
+/**
+ * The file state backend is a state backend that stores the state of streaming jobs in a file system.
+ * 
+ * <p>The state backend has one core directory into which it puts all checkpoint data. Inside that
+ * directory, it creates a directory per job, inside which each checkpoint gets a directory, with
+ * files for each state, for example:
+ * 
+ * {@code hdfs://namenode:port/flink-checkpoints/<job-id>/chk-17/6ba7b810-9dad-11d1-80b4-00c04fd430c8 }
+ */
+public class FsStateBackend extends StateBackend<FsStateBackend> {
+
+	private static final long serialVersionUID = -8191916350224044011L;
+	
+	private static final Logger LOG = LoggerFactory.getLogger(FsStateBackend.class);
+	
+	
+	/** The path to the directory for the checkpoint data, including the file system
+	 * description via scheme and optional authority */
+	private final Path basePath;
+	
+	/** The directory (job specific) into this initialized instance of the backend stores its data */
+	private transient Path checkpointDirectory;
+	
+	/** Cached handle to the file system for file operations */
+	private transient FileSystem filesystem;
+
+
+	/**
+	 * Creates a new state backend that stores its checkpoint data in the file system and location
+	 * defined by the given URI.
+	 *
+	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+	 * must be accessible via {@link FileSystem#get(URI)}.
+	 *
+	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+	 * (host and port), or that the Hadoop configuration that describes that information must be in the
+	 * classpath.
+	 *
+	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+	 *                          and the path to teh checkpoint data directory.
+	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+	 */
+	public FsStateBackend(String checkpointDataUri) throws IOException {
+		this(new Path(checkpointDataUri));
+	}
+
+	/**
+	 * Creates a new state backend that stores its checkpoint data in the file system and location
+	 * defined by the given URI.
+	 *
+	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+	 * must be accessible via {@link FileSystem#get(URI)}.
+	 *
+	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+	 * (host and port), or that the Hadoop configuration that describes that information must be in the
+	 * classpath.
+	 *
+	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+	 *                          and the path to teh checkpoint data directory.
+	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+	 */
+	public FsStateBackend(Path checkpointDataUri) throws IOException {
+		this(checkpointDataUri.toUri());
+	}
+
+	/**
+	 * Creates a new state backend that stores its checkpoint data in the file system and location
+	 * defined by the given URI.
+	 * 
+	 * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+	 * must be accessible via {@link FileSystem#get(URI)}.
+	 * 
+	 * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+	 * (host and port), or that the Hadoop configuration that describes that information must be in the
+	 * classpath.
+	 * 
+	 * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+	 *                          and the path to teh checkpoint data directory.
+	 * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+	 */
+	public FsStateBackend(URI checkpointDataUri) throws IOException {
+		final String scheme = checkpointDataUri.getScheme();
+		final String path = checkpointDataUri.getPath();
+		
+		// some validity checks
+		if (scheme == null) {
+			throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " +
+					"Please specify the file system scheme explicitly in the URI.");
+		}
+		if (path == null) {
+			throw new IllegalArgumentException("The path to store the checkpoint data in is null. " +
+					"Please specify a directory path for the checkpoint data.");
+		}
+		if (path.length() == 0 || path.equals("/")) {
+			throw new IllegalArgumentException("Cannot use the root directory for checkpoints.");
+		}
+		
+		// we do a bit of work to make sure that the URI for the filesystem refers to exactly the same
+		// (distributed) filesystem on all hosts and includes full host/port information, even if the
+		// original URI did not include that. We count on the filesystem loading from the configuration
+		// to fill in the missing data.
+		
+		// try to grab the file system for this path/URI
+		this.filesystem = FileSystem.get(checkpointDataUri);
+		if (this.filesystem == null) {
+			throw new IOException("Could not find a file system for the given scheme in the available configurations.");
+		}
+
+		URI fsURI = this.filesystem.getUri();
+		try {
+			URI baseURI = new URI(fsURI.getScheme(), fsURI.getAuthority(), path, null, null);
+			this.basePath = new Path(baseURI);
+		}
+		catch (URISyntaxException e) {
+			throw new IOException(
+					String.format("Cannot create file system URI for checkpointDataUri %s and filesystem URI %s", 
+							checkpointDataUri, fsURI), e);
+		}
+	}
+
+	/**
+	 * Gets the base directory where all state-containing files are stored.
+	 * The job specific directory is created inside this directory.
+	 * 
+	 * @return The base directory.
+	 */
+	public Path getBasePath() {
+		return basePath;
+	}
+
+	/**
+	 * Gets the directory where this state backend stores its checkpoint data. Will be null if
+	 * the state backend has not been initialized.
+	 * 
+	 * @return The directory where this state backend stores its checkpoint data.
+	 */
+	public Path getCheckpointDirectory() {
+		return checkpointDirectory;
+	}
+
+	/**
+	 * Checks whether this state backend is initialized. Note that initialization does not carry
+	 * across serialization. After each serialization, the state backend needs to be initialized.
+	 * 
+	 * @return True, if the file state backend has been initialized, false otherwise.
+	 */
+	public boolean isInitialized() {
+		return filesystem != null && checkpointDirectory != null; 
+	}
+
+	/**
+	 * Gets the file system handle for the file system that stores the state for this backend.
+	 * 
+	 * @return This backend's file system handle.
+	 */
+	public FileSystem getFileSystem() {
+		if (filesystem != null) {
+			return filesystem;
+		}
+		else {
+			throw new IllegalStateException("State backend has not been initialized.");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  initialization and cleanup
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public void initializeForJob(JobID jobId) throws Exception {
+		Path dir = new Path(basePath, jobId.toString());
+		
+		LOG.info("Initializing file state backend to URI " + dir);
+		
+		filesystem = basePath.getFileSystem();
+		filesystem.mkdirs(dir);
+
+		checkpointDirectory = dir;
+	}
+
+	@Override
+	public void disposeAllStateForCurrentJob() throws Exception {
+		FileSystem fs = this.filesystem;
+		Path dir = this.checkpointDirectory;
+		
+		if (fs != null && dir != null) {
+			this.filesystem = null;
+			this.checkpointDirectory = null;
+			fs.delete(dir, true);
+		}
+		else {
+			throw new IllegalStateException("state backend has not been initialized");
+		}
+	}
+
+	@Override
+	public void close() throws Exception {}
+
+	// ------------------------------------------------------------------------
+	//  state backend operations
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public <K, V> FsHeapKvState<K, V> createKvState(
+			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) throws Exception {
+		return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, this);
+	}
+
+	@Override
+	public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+			S state, long checkpointID, long timestamp) throws Exception
+	{
+		checkFileSystemInitialized();
+
+		// make sure the directory for that specific checkpoint exists
+		final Path checkpointDir = createCheckpointDirPath(checkpointID);
+		filesystem.mkdirs(checkpointDir);
+
+		
+		Exception latestException = null;
+
+		for (int attempt = 0; attempt < 10; attempt++) {
+			Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
+			FSDataOutputStream outStream;
+			try {
+				outStream = filesystem.create(targetPath, false);
+			}
+			catch (Exception e) {
+				latestException = e;
+				continue;
+			}
+
+			ObjectOutputStream os = new ObjectOutputStream(outStream);
+			os.writeObject(state);
+			os.close();
+			return new FileSerializableStateHandle<S>(targetPath);
+		}
+		
+		throw new Exception("Could not open output stream for state backend", latestException);
+	}
+	
+	@Override
+	public FsCheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp) throws Exception {
+		checkFileSystemInitialized();
+		
+		final Path checkpointDir = createCheckpointDirPath(checkpointID);
+		filesystem.mkdirs(checkpointDir);
+		
+		Exception latestException = null;
+		
+		for (int attempt = 0; attempt < 10; attempt++) {
+			Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
+			try {
+				FSDataOutputStream outStream = filesystem.create(targetPath, false);
+				return new FsCheckpointStateOutputStream(outStream, targetPath, filesystem);
+			}
+			catch (Exception e) {
+				latestException = e;
+			}
+		}
+		throw new Exception("Could not open output stream for state backend", latestException);
+	}
+	
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private void checkFileSystemInitialized() throws IllegalStateException {
+		if (filesystem == null || checkpointDirectory == null) {
+			throw new IllegalStateException("filesystem has not been re-initialized after deserialization");
+		}
+	}
+	
+	private Path createCheckpointDirPath(long checkpointID) {
+		return new Path(checkpointDirectory, "chk-" + checkpointID);
+	}
+	
+	@Override
+	public String toString() {
+		return checkpointDirectory == null ?
+			"File State Backend @ " + basePath : 
+			"File State Backend (initialized) @ " + checkpointDirectory;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Output stream for state checkpointing
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A CheckpointStateOutputStream that writes into a file and returns the path to that file upon
+	 * closing.
+	 */
+	public static final class FsCheckpointStateOutputStream extends CheckpointStateOutputStream {
+
+		private final FSDataOutputStream outStream;
+		
+		private final Path filePath;
+		
+		private final FileSystem fs;
+		
+		private boolean closed;
+
+		FsCheckpointStateOutputStream(FSDataOutputStream outStream, Path filePath, FileSystem fs) {
+			this.outStream = outStream;
+			this.filePath = filePath;
+			this.fs = fs;
+		}
+
+
+		@Override
+		public void write(int b) throws IOException {
+			outStream.write(b);
+		}
+
+		@Override
+		public void write(byte[] b, int off, int len) throws IOException {
+			outStream.write(b, off, len);
+		}
+
+		@Override
+		public void flush() throws IOException {
+			outStream.flush();
+		}
+
+		/**
+		 * If the stream is only closed, we remove the produced file (cleanup through the auto close
+		 * feature, for example). This method throws no exception if the deletion fails, but only
+		 * logs the error.
+		 */
+		@Override
+		public void close() {
+			synchronized (this) {
+				if (!closed) {
+					closed = true;
+					try {
+						outStream.close();
+						fs.delete(filePath, false);
+						
+						// attempt to delete the parent (will fail and be ignored if the parent has more files)
+						try {
+							fs.delete(filePath.getParent(), false);
+						} catch (IOException ignored) {}
+					}
+					catch (Exception e) {
+						LOG.warn("Cannot delete closed and discarded state stream to " + filePath, e);
+					}
+				}
+			}
+		}
+
+		@Override
+		public FileStreamStateHandle closeAndGetHandle() throws IOException {
+			return new FileStreamStateHandle(closeAndGetPath());
+		}
+
+		/**
+		 * Closes the stream and returns the path to the file that contains the stream's data.
+		 * @return The path to the file that contains the stream's data.
+		 * @throws IOException Thrown if the stream cannot be successfully closed.
+		 */
+		public Path closeAndGetPath() throws IOException {
+			synchronized (this) {
+				if (!closed) {
+					closed = true;
+					outStream.close();
+					return filePath;
+				}
+				else {
+					throw new IOException("Stream has already been closed and discarded.");
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
new file mode 100644
index 0000000..e687f7f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.filesystem;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateBackendFactory;
+
+/**
+ * A factory that creates an {@link org.apache.flink.runtime.state.filesystem.FsStateBackend}
+ * from a configuration.
+ */
+public class FsStateBackendFactory implements StateBackendFactory<FsStateBackend> {
+	
+	/** The key under which the config stores the directory where checkpoints should be stored */
+	public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
+	
+	
+	@Override
+	public FsStateBackend createFromConfig(Configuration config) throws Exception {
+		String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+
+		if (checkpointDirURI == null) {
+			throw new IllegalConfigurationException(
+					"Cannot create the file system state backend: The configuration does not specify the " +
+							"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
+		}
+		
+		try {
+			Path path = new Path(checkpointDirURI);
+			return new FsStateBackend(path);
+		}
+		catch (IllegalArgumentException e) {
+			throw new Exception("Cannot initialize File System State Backend with URI '"
+					+ checkpointDirURI + '.', e);
+		}
+		
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java
new file mode 100644
index 0000000..29762f7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java
@@ -0,0 +1,52 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+
+/**
+ * A state handle that contains stream state in a byte array.
+ */
+public final class ByteStreamStateHandle implements StreamStateHandle {
+
+	private static final long serialVersionUID = -5280226231200217594L;
+	
+	/** the state data */
+	private final byte[] data;
+
+	/**
+	 * Creates a new ByteStreamStateHandle containing the given data.
+	 * 
+	 * @param data The state data.
+	 */
+	public ByteStreamStateHandle(byte[] data) {
+		this.data = data;
+	}
+
+	@Override
+	public InputStream getState(ClassLoader userCodeClassLoader) {
+		return new ByteArrayInputStream(data);
+	}
+
+	@Override
+	public void discardState() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemHeapKvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemHeapKvState.java
new file mode 100644
index 0000000..96cb440
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemHeapKvState.java
@@ -0,0 +1,52 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.util.DataOutputSerializer;
+import org.apache.flink.runtime.state.AbstractHeapKvState;
+
+import java.util.HashMap;
+
+/**
+ * Heap-backed key/value state that is snapshotted into a serialized memory copy.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public class MemHeapKvState<K, V> extends AbstractHeapKvState<K, V, MemoryStateBackend> {
+	
+	public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
+		super(keySerializer, valueSerializer, defaultValue);
+	}
+
+	public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+							V defaultValue, HashMap<K, V> state) {
+		super(keySerializer, valueSerializer, defaultValue, state);
+	}
+	
+	@Override
+	public MemoryHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
+		DataOutputSerializer ser = new DataOutputSerializer(Math.max(size() * 16, 16));
+		writeStateToOutputView(ser);
+		byte[] bytes = ser.getCopyOfBuffer();
+		
+		return new MemoryHeapKvStateSnapshot<K, V>(getKeySerializer(), getValueSerializer(), bytes, size());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryHeapKvStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryHeapKvStateSnapshot.java
new file mode 100644
index 0000000..1b03def
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryHeapKvStateSnapshot.java
@@ -0,0 +1,102 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.runtime.state.KvStateSnapshot;
+
+import java.util.HashMap;
+
+/**
+ * A snapshot of a {@link MemHeapKvState} for a checkpoint. The data is stored in a heap byte
+ * array, in serialized form.
+ * 
+ * @param <K> The type of the key in the snapshot state.
+ * @param <V> The type of the value in the snapshot state.
+ */
+public class MemoryHeapKvStateSnapshot<K, V> implements KvStateSnapshot<K, V, MemoryStateBackend> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	/** Name of the key serializer class */
+	private final String keySerializerClassName;
+
+	/** Name of the value serializer class */
+	private final String valueSerializerClassName;
+	
+	/** The serialized data of the state key/value pairs */
+	private final byte[] data;
+	
+	/** The number of key/value pairs */
+	private final int numEntries;
+
+	/**
+	 * Creates a new heap memory state snapshot.
+	 *
+	 * @param keySerializer The serializer for the keys.
+	 * @param valueSerializer The serializer for the values.
+	 * @param data The serialized data of the state key/value pairs
+	 * @param numEntries The number of key/value pairs
+	 */
+	public MemoryHeapKvStateSnapshot(TypeSerializer<K> keySerializer,
+						TypeSerializer<V> valueSerializer, byte[] data, int numEntries) {
+		this.keySerializerClassName = keySerializer.getClass().getName();
+		this.valueSerializerClassName = valueSerializer.getClass().getName();
+		this.data = data;
+		this.numEntries = numEntries;
+	}
+
+
+	@Override
+	public MemHeapKvState<K, V> restoreState(
+			MemoryStateBackend stateBackend,
+			final TypeSerializer<K> keySerializer,
+			final TypeSerializer<V> valueSerializer,
+			V defaultValue,
+			ClassLoader classLoader) throws Exception {
+
+		// validity checks
+		if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
+			!valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
+				throw new IllegalArgumentException(
+						"Cannot restore the state from the snapshot with the given serializers. " +
+						"State (K/V) was serialized with (" + valueSerializerClassName + 
+						"/" + keySerializerClassName + ")");
+		}
+		
+		// restore state
+		HashMap<K, V> stateMap = new HashMap<>(numEntries);
+		DataInputDeserializer in = new DataInputDeserializer(data, 0, data.length);
+		
+		for (int i = 0; i < numEntries; i++) {
+			K key = keySerializer.deserialize(in);
+			V value = valueSerializer.deserialize(in);
+			stateMap.put(key, value);
+		}
+		
+		return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap);
+	}
+
+	/**
+	 * Discarding the heap state is a no-op.
+	 */
+	@Override
+	public void discardState() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
new file mode 100644
index 0000000..8d297d4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.memory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * A {@link StateBackend} that stores all its data and checkpoints in memory and has no
+ * capabilities to spill to disk. Checkpoints are serialized and the serialized data is
+ * transferred 
+ */
+public class MemoryStateBackend extends StateBackend<MemoryStateBackend> {
+
+	private static final long serialVersionUID = 4109305377809414635L;
+	
+	/** The default maximal size that the snapshotted memory state may have (5 MiBytes) */
+	private static final int DEFAULT_MAX_STATE_SIZE = 5 * 1024 * 1024;
+	
+	/** The maximal size that the snapshotted memory state may have */
+	private final int maxStateSize;
+
+	/**
+	 * Creates a new memory state backend that accepts states whose serialized forms are
+	 * up to the default state size (5 MB).
+	 */
+	public MemoryStateBackend() {
+		this(DEFAULT_MAX_STATE_SIZE);
+	}
+
+	/**
+	 * Creates a new memory state backend that accepts states whose serialized forms are
+	 * up to the given number of bytes.
+	 * 
+	 * @param maxStateSize The maximal size of the serialized state
+	 */
+	public MemoryStateBackend(int maxStateSize) {
+		this.maxStateSize = maxStateSize;
+	}
+
+	// ------------------------------------------------------------------------
+	//  initialization and cleanup
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void initializeForJob(JobID job) {
+		// nothing to do here
+	}
+
+	@Override
+	public void disposeAllStateForCurrentJob() {
+		// nothing to do here, GC will do it
+	}
+
+	@Override
+	public void close() throws Exception {}
+
+	// ------------------------------------------------------------------------
+	//  State backend operations
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public <K, V> MemHeapKvState<K, V> createKvState(
+			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
+		return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue);
+	}
+	
+	/**
+	 * Serialized the given state into bytes using Java serialization and creates a state handle that
+	 * can re-create that state.
+	 * 
+	 * @param state The state to checkpoint.
+	 * @param checkpointID The ID of the checkpoint.
+	 * @param timestamp The timestamp of the checkpoint.
+	 * @param <S> The type of the state.
+	 * 
+	 * @return A state handle that contains the given state serialized as bytes.
+	 * @throws Exception Thrown, if the serialization fails.
+	 */
+	@Override
+	public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+			S state, long checkpointID, long timestamp) throws Exception
+	{
+		SerializedStateHandle<S> handle = new SerializedStateHandle<>(state);
+		checkSize(handle.getSizeOfSerializedState(), maxStateSize);
+		return new SerializedStateHandle<S>(state);
+	}
+
+	@Override
+	public CheckpointStateOutputStream createCheckpointStateOutputStream(
+			long checkpointID, long timestamp) throws Exception
+	{
+		return new MemoryCheckpointOutputStream(maxStateSize);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "MemoryStateBackend (data in heap memory / checkpoints to JobManager)";
+	}
+
+	static void checkSize(int size, int maxSize) throws IOException {
+		if (size > maxSize) {
+			throw new IOException(
+					"Size of the state is larger than the maximum permitted memory-backed state. Size="
+							+ size + " , maxSize=" + maxSize
+							+ " . Consider using a different state backend, like the File System State backend.");
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A CheckpointStateOutputStream that writes into a byte array.
+	 */
+	public static final class MemoryCheckpointOutputStream extends CheckpointStateOutputStream {
+		
+		private final ByteArrayOutputStream os = new ByteArrayOutputStream();
+		
+		private final int maxSize;
+		
+		private boolean closed;
+
+		public MemoryCheckpointOutputStream(int maxSize) {
+			this.maxSize = maxSize;
+		}
+
+		@Override
+		public void write(int b) {
+			os.write(b);
+		}
+
+		@Override
+		public void write(byte[] b, int off, int len) {
+			os.write(b, off, len);
+		}
+
+		// --------------------------------------------------------------------
+
+		@Override
+		public void close() {
+			closed = true;
+			os.reset();
+		}
+
+		@Override
+		public StreamStateHandle closeAndGetHandle() throws IOException {
+			return new ByteStreamStateHandle(closeAndGetBytes());
+		}
+
+		/**
+		 * Closes the stream and returns the byte array containing the stream's data.
+		 * @return The byte array containing the stream's data.
+		 * @throws IOException Thrown if the size of the data exceeds the maximal 
+		 */
+		public byte[] closeAndGetBytes() throws IOException {
+			if (!closed) {
+				checkSize(os.size(), maxSize);
+				byte[] bytes = os.toByteArray();
+				close();
+				return bytes;
+			}
+			else {
+				throw new IllegalStateException("stream has already been closed");
+			}
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Static default instance
+	// ------------------------------------------------------------------------
+	
+	/** The default instance of this state backend, using the default maximal state size */
+	private static final MemoryStateBackend DEFAULT_INSTANCE = new MemoryStateBackend();
+
+	/**
+	 * Gets the default instance of this state backend, using the default maximal state size.
+	 * @return The default instance of this state backend.
+	 */
+	public static MemoryStateBackend defaultInstance() {
+		return DEFAULT_INSTANCE;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/SerializedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/SerializedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/SerializedStateHandle.java
new file mode 100644
index 0000000..c488dc9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/SerializedStateHandle.java
@@ -0,0 +1,49 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+
+/**
+ * A state handle that represents its state in serialized form as bytes.
+ *
+ * @param <T> The type of state represented by this state handle.
+ */
+public class SerializedStateHandle<T> extends SerializedValue<T> implements StateHandle<T> {
+	
+	private static final long serialVersionUID = 4145685722538475769L;
+
+	public SerializedStateHandle(T value) throws IOException {
+		super(value);
+	}
+	
+	@Override
+	public T getState(ClassLoader classLoader) throws Exception {
+		return deserializeValue(classLoader);
+	}
+
+	/**
+	 * Discarding heap-memory backed state is a no-op, so this method does nothing.
+	 */
+	@Override
+	public void discardState() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
index 79b9b7e..a32fc65 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
@@ -34,11 +34,14 @@ import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
 import org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
-import org.apache.flink.runtime.state.StateHandleProvider;
-import org.apache.flink.runtime.state.StateHandleProviderFactory;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
+import org.apache.flink.runtime.zookeeper.filesystem.FileSystemStateStorageHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.io.Serializable;
+
 import static com.google.common.base.Preconditions.checkNotNull;
 
 public class ZooKeeperUtils {
@@ -170,7 +173,7 @@ public class ZooKeeperUtils {
 		String latchPath = configuration.getString(ConfigConstants.ZOOKEEPER_LATCH_PATH,
 				ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH);
 		String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
-				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
+			ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
 
 		return new ZooKeeperLeaderElectionService(client, latchPath, leaderPath);
 	}
@@ -188,8 +191,7 @@ public class ZooKeeperUtils {
 
 		checkNotNull(configuration, "Configuration");
 
-		StateHandleProvider<SubmittedJobGraph> stateHandleProvider =
-				StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration);
+		StateStorageHelper<SubmittedJobGraph> stateStorage = createFileSystemStateStorage(configuration, "submittedJobGraph");
 
 		// ZooKeeper submitted jobs root dir
 		String zooKeeperSubmittedJobsPath = configuration.getString(
@@ -197,7 +199,7 @@ public class ZooKeeperUtils {
 				ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
 
 		return new ZooKeeperSubmittedJobGraphStore(
-				client, zooKeeperSubmittedJobsPath, stateHandleProvider);
+				client, zooKeeperSubmittedJobsPath, stateStorage);
 	}
 
 	/**
@@ -219,21 +221,23 @@ public class ZooKeeperUtils {
 
 		checkNotNull(configuration, "Configuration");
 
-		StateHandleProvider<CompletedCheckpoint> stateHandleProvider =
-				StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration);
+		String checkpointsPath = configuration.getString(
+			ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH,
+			ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
 
-		String completedCheckpointsPath = configuration.getString(
-				ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH,
-				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
 
-		completedCheckpointsPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId);
+		StateStorageHelper<CompletedCheckpoint> stateStorage = createFileSystemStateStorage(
+			configuration,
+			"completedCheckpoint");
+
+		checkpointsPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId);
 
 		return new ZooKeeperCompletedCheckpointStore(
 				maxNumberOfCheckpointsToRetain,
 				userClassLoader,
 				client,
-				completedCheckpointsPath,
-				stateHandleProvider);
+				checkpointsPath,
+				stateStorage);
 	}
 
 	/**
@@ -259,6 +263,30 @@ public class ZooKeeperUtils {
 	}
 
 	/**
+	 * Creates a {@link FileSystemStateStorageHelper} instance.
+	 *
+	 * @param configuration {@link Configuration} object
+	 * @param prefix Prefix for the created files
+	 * @param <T> Type of the state objects
+	 * @return {@link FileSystemStateStorageHelper} instance
+	 * @throws IOException
+	 */
+	private static <T extends Serializable> FileSystemStateStorageHelper<T> createFileSystemStateStorage(
+			Configuration configuration,
+			String prefix) throws IOException {
+
+		String rootPath = configuration.getString(
+			ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
+
+		if (rootPath.equals("")) {
+			throw new IllegalConfigurationException("Missing recovery path. Specify via " +
+				"configuration key '" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "'.");
+		} else {
+			return new FileSystemStateStorageHelper<T>(rootPath, prefix);
+		}
+	}
+
+	/**
 	 * Private constructor to prevent instantiation.
 	 */
 	private ZooKeeperUtils() {

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
new file mode 100644
index 0000000..d18cace
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
@@ -0,0 +1,41 @@
+/*
+ * 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.zookeeper;
+
+import org.apache.flink.runtime.state.StateHandle;
+
+import java.io.Serializable;
+
+/**
+ * State storage helper which is used by {@ZooKeeperStateHandleStore} to persiste state before
+ * the state handle is written to ZooKeeper.
+ *
+ * @param <T>
+ */
+public interface StateStorageHelper<T extends Serializable> {
+
+	/**
+	 * Stores the given state and returns a state handle to it.
+	 *
+	 * @param state State to be stored
+	 * @return State handle to the stored state
+	 * @throws Exception
+	 */
+	StateHandle<T> store(T state) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
index 936fe1b..6073a39 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
@@ -23,12 +23,14 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
@@ -65,11 +67,12 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 public class ZooKeeperStateHandleStore<T extends Serializable> {
 
+	public static Logger LOG = LoggerFactory.getLogger(ZooKeeperStateHandleStore.class);
+
 	/** Curator ZooKeeper client */
 	private final CuratorFramework client;
 
-	/** State handle provider */
-	private final StateHandleProvider<T> stateHandleProvider;
+	private final StateStorageHelper<T> storage;
 
 	/**
 	 * Creates a {@link ZooKeeperStateHandleStore}.
@@ -78,14 +81,13 @@ public class ZooKeeperStateHandleStore<T extends Serializable> {
 	 *                            expected that the client's namespace ensures that the root
 	 *                            path is exclusive for all state handles managed by this
 	 *                            instance, e.g. <code>client.usingNamespace("/stateHandles")</code>
-	 * @param stateHandleProvider The state handle provider for the state
 	 */
 	public ZooKeeperStateHandleStore(
-			CuratorFramework client,
-			StateHandleProvider<T> stateHandleProvider) {
+		CuratorFramework client,
+		StateStorageHelper storage) throws IOException {
 
 		this.client = checkNotNull(client, "Curator client");
-		this.stateHandleProvider = checkNotNull(stateHandleProvider, "State handle provider");
+		this.storage = checkNotNull(storage, "State storage");
 	}
 
 	/**
@@ -112,12 +114,14 @@ public class ZooKeeperStateHandleStore<T extends Serializable> {
 	 * @return Created {@link StateHandle}
 	 * @throws Exception If a ZooKeeper or state handle operation fails
 	 */
-	public StateHandle<T> add(String pathInZooKeeper, T state, CreateMode createMode) throws Exception {
+	public StateHandle<T> add(
+			String pathInZooKeeper,
+			T state,
+			CreateMode createMode) throws Exception {
 		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
 		checkNotNull(state, "State");
 
-		// Create the state handle. Nothing persisted yet.
-		StateHandle<T> stateHandle = stateHandleProvider.createStateHandle(state);
+		StateHandle<T> stateHandle = storage.store(state);
 
 		boolean success = false;
 
@@ -159,7 +163,7 @@ public class ZooKeeperStateHandleStore<T extends Serializable> {
 
 		StateHandle<T> oldStateHandle = get(pathInZooKeeper);
 
-		StateHandle<T> stateHandle = stateHandleProvider.createStateHandle(state);
+		StateHandle<T> stateHandle = storage.store(state);
 
 		boolean success = false;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/filesystem/FileSystemStateStorageHelper.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/filesystem/FileSystemStateStorageHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/filesystem/FileSystemStateStorageHelper.java
new file mode 100644
index 0000000..d6b69e4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/filesystem/FileSystemStateStorageHelper.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.zookeeper.filesystem;
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.filesystem.FileSerializableStateHandle;
+import org.apache.flink.runtime.util.FileUtils;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
+
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+/**
+ * {@link StateStorageHelper} implementation which stores the state in the given filesystem path.
+ *
+ * @param <T>
+ */
+public class FileSystemStateStorageHelper<T extends Serializable> implements StateStorageHelper<T> {
+
+	private final Path rootPath;
+
+	private final String prefix;
+
+	private final FileSystem fs;
+
+	public FileSystemStateStorageHelper(String rootPath, String prefix) throws IOException {
+		this(new Path(rootPath), prefix);
+	}
+
+	public FileSystemStateStorageHelper(Path rootPath, String prefix) throws IOException {
+		this.rootPath = Preconditions.checkNotNull(rootPath, "Root path");
+		this.prefix = Preconditions.checkNotNull(prefix, "Prefix");
+
+		fs = FileSystem.get(rootPath.toUri());
+	}
+
+	@Override
+	public StateHandle<T> store(T state) throws Exception {
+		Exception latestException = null;
+
+		for (int attempt = 0; attempt < 10; attempt++) {
+			Path filePath = getNewFilePath();
+			FSDataOutputStream outStream;
+			try {
+				outStream = fs.create(filePath, false);
+			}
+			catch (Exception e) {
+				latestException = e;
+				continue;
+			}
+
+			try(ObjectOutputStream os = new ObjectOutputStream(outStream)) {
+				os.writeObject(state);
+			}
+
+			return new FileSerializableStateHandle<>(filePath);
+		}
+
+		throw new Exception("Could not open output stream for state backend", latestException);
+	}
+
+	private Path getNewFilePath() {
+		return new Path(rootPath, FileUtils.getRandomFilename(prefix));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/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 ebc0ea9..d9b69ad 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
@@ -1542,30 +1542,25 @@ object JobManager {
       }
     }
 
-    val webMonitor: Option[WebMonitor] =
-      if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
-        val address = AkkaUtils.getAddress(jobManagerSystem)
+    val address = AkkaUtils.getAddress(jobManagerSystem)
 
-        configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
-        configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
+    configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
+    configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
 
-        // start the job manager web frontend
-        if (configuration.getBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, false)) {
-          val leaderRetrievalService = LeaderRetrievalUtils
-            .createLeaderRetrievalService(configuration)
+    val webMonitor: Option[WebMonitor] =
+      if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
+        LOG.info("Starting JobManger web frontend")
+        val leaderRetrievalService = LeaderRetrievalUtils
+          .createLeaderRetrievalService(configuration)
 
-          LOG.info("Starting NEW JobManger web frontend")
-          // start the new web frontend. we need to load this dynamically
-          // because it is not in the same project/dependencies
-          Some(startWebRuntimeMonitor(configuration, leaderRetrievalService, jobManagerSystem))
-        }
-        else {
-          LOG.info("Starting JobManger web frontend")
+        // start the web frontend. we need to load this dynamically
+        // because it is not in the same project/dependencies
+        val webServer = WebMonitorUtils.startWebRuntimeMonitor(
+          configuration,
+          leaderRetrievalService,
+          jobManagerSystem)
 
-          // The old web frontend does not work with recovery mode
-          val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(configuration)
-          Some(new WebInfoServer(configuration, leaderRetrievalService, jobManagerSystem))
-        }
+        Option(webServer)
       }
       else {
         None
@@ -1624,16 +1619,8 @@ object JobManager {
         monitor =>
           val jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(configuration)
           monitor.start(jobManagerAkkaUrl)
-        LOG.info("Starting JobManger web frontend")
-        // start the web frontend. we need to load this dynamically
-        // because it is not in the same project/dependencies
-        val webServer = WebMonitorUtils.startWebRuntimeMonitor(
-          configuration,
-          leaderRetrievalService,
-          jobManagerSystem)
       }
 
-
       (jobManagerSystem, jobManager, archive, webMonitor)
     }
     catch {

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
index 4c6ddfd..dc6f550 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.runtime.state.LocalStateHandle;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -56,8 +58,12 @@ public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpoint
 			ClassLoader userLoader) throws Exception {
 
 		return new ZooKeeperCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userLoader,
-				ZooKeeper.createClient(), CheckpointsPath, new LocalStateHandle
-				.LocalStateHandleProvider<CompletedCheckpoint>());
+			ZooKeeper.createClient(), CheckpointsPath, new StateStorageHelper<CompletedCheckpoint>() {
+			@Override
+			public StateHandle<CompletedCheckpoint> store(CompletedCheckpoint state) throws Exception {
+				return new LocalStateHandle<>(state);
+			}
+		});
 	}
 
 	// ---------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
index 4df8afb..ea4195c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.execution.librarycache;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -28,9 +27,9 @@ import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.blob.BlobServer;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobmanager.RecoveryMode;
-import org.junit.After;
-import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -46,23 +45,8 @@ import static org.junit.Assert.assertEquals;
 
 public class BlobLibraryCacheRecoveryITCase {
 
-	private File recoveryDir;
-
-	@Before
-	public void setUp() throws Exception {
-		recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir");
-		if (!recoveryDir.exists() && !recoveryDir.mkdirs()) {
-			throw new IllegalStateException("Failed to create temp directory for test");
-		}
-	}
-
-	@After
-	public void cleanUp() throws Exception {
-		if (recoveryDir != null) {
-			FileUtils.deleteDirectory(recoveryDir);
-		}
-	}
-
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
 	/**
 	 * Tests that with {@link RecoveryMode#ZOOKEEPER} distributed JARs are recoverable from any
 	 * participating BlobLibraryCacheManager.
@@ -81,7 +65,7 @@ public class BlobLibraryCacheRecoveryITCase {
 			Configuration config = new Configuration();
 			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
 			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-			config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, recoveryDir.getPath());
+			config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, temporaryFolder.getRoot().getAbsolutePath());
 
 			for (int i = 0; i < server.length; i++) {
 				server[i] = new BlobServer(config);
@@ -170,7 +154,7 @@ public class BlobLibraryCacheRecoveryITCase {
 		}
 
 		// Verify everything is clean
-		File[] recoveryFiles = recoveryDir.listFiles();
+		File[] recoveryFiles = temporaryFolder.getRoot().listFiles();
 		assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
index 861a713..356ba36 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
@@ -24,7 +24,9 @@ import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener;
-import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider;
+import org.apache.flink.runtime.state.LocalStateHandle;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -54,8 +56,13 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
 
 	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
 
-	private final static LocalStateHandleProvider<SubmittedJobGraph> StateHandleProvider =
-			new LocalStateHandleProvider<>();
+	private final static StateStorageHelper<SubmittedJobGraph> localStateStorage = new StateStorageHelper<SubmittedJobGraph>() {
+		@Override
+		public StateHandle<SubmittedJobGraph> store(SubmittedJobGraph state) throws Exception {
+			return new LocalStateHandle<>(state);
+		}
+	};
+
 
 	@AfterClass
 	public static void tearDown() throws Exception {
@@ -72,8 +79,9 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
 	@Test
 	public void testPutAndRemoveJobGraph() throws Exception {
 		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
-				ZooKeeper.createClient(), "/testPutAndRemoveJobGraph",
-				StateHandleProvider);
+			ZooKeeper.createClient(),
+			"/testPutAndRemoveJobGraph",
+			localStateStorage);
 
 		try {
 			SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class);
@@ -125,7 +133,7 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
 	@Test
 	public void testRecoverJobGraphs() throws Exception {
 		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
-				ZooKeeper.createClient(), "/testRecoverJobGraphs", StateHandleProvider);
+				ZooKeeper.createClient(), "/testRecoverJobGraphs", localStateStorage);
 
 		try {
 			SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class);
@@ -175,10 +183,10 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
 
 		try {
 			jobGraphs = new ZooKeeperSubmittedJobGraphStore(
-					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider);
+					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", localStateStorage);
 
 			otherJobGraphs = new ZooKeeperSubmittedJobGraphStore(
-					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider);
+					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", localStateStorage);
 
 
 			SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0);
@@ -234,10 +242,10 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
 	@Test(expected = IllegalStateException.class)
 	public void testUpdateJobGraphYouDidNotGetOrAdd() throws Exception {
 		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
-				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider);
+				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", localStateStorage);
 
 		ZooKeeperSubmittedJobGraphStore otherJobGraphs = new ZooKeeperSubmittedJobGraphStore(
-				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider);
+				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", localStateStorage);
 
 		jobGraphs.start(null);
 		otherJobGraphs.start(null);


[23/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot
new file mode 100644
index 0000000..33b2bb8
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot differ


[31/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
new file mode 100644
index 0000000..481fb98
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
@@ -0,0 +1,423 @@
+/*
+ * 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.state;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.KvStateSnapshot;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FileStreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+public class FileStateBackendTest {
+	
+	@Test
+	public void testSetupAndSerialization() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+		try {
+			final String backendDir = localFileUri(tempDir);
+			FsStateBackend originalBackend = new FsStateBackend(backendDir);
+			
+			assertFalse(originalBackend.isInitialized());
+			assertEquals(new URI(backendDir), originalBackend.getBasePath().toUri());
+			assertNull(originalBackend.getCheckpointDirectory());
+			
+			// serialize / copy the backend
+			FsStateBackend backend = CommonTestUtils.createCopySerializable(originalBackend);
+			assertFalse(backend.isInitialized());
+			assertEquals(new URI(backendDir), backend.getBasePath().toUri());
+			assertNull(backend.getCheckpointDirectory());
+			
+			// no file operations should be possible right now
+			try {
+				backend.checkpointStateSerializable("exception train rolling in", 2L, System.currentTimeMillis());
+				fail("should fail with an exception");
+			} catch (IllegalStateException e) {
+				// supreme!
+			}
+			
+			backend.initializeForJob(new JobID());
+			assertNotNull(backend.getCheckpointDirectory());
+			
+			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+			assertTrue(checkpointDir.exists());
+			assertTrue(isDirectoryEmpty(checkpointDir));
+			
+			backend.disposeAllStateForCurrentJob();
+			assertNull(backend.getCheckpointDirectory());
+			
+			assertTrue(isDirectoryEmpty(tempDir));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			deleteDirectorySilently(tempDir);
+		}
+	}
+	
+	@Test
+	public void testSerializableState() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+		try {
+			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+			backend.initializeForJob(new JobID());
+
+			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+			String state1 = "dummy state";
+			String state2 = "row row row your boat";
+			Integer state3 = 42;
+			
+			StateHandle<String> handle1 = backend.checkpointStateSerializable(state1, 439568923746L, System.currentTimeMillis());
+			StateHandle<String> handle2 = backend.checkpointStateSerializable(state2, 439568923746L, System.currentTimeMillis());
+			StateHandle<Integer> handle3 = backend.checkpointStateSerializable(state3, 439568923746L, System.currentTimeMillis());
+
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			assertEquals(state1, handle1.getState(getClass().getClassLoader()));
+			handle1.discardState();
+			
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			assertEquals(state2, handle2.getState(getClass().getClassLoader()));
+			handle2.discardState();
+			
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			assertEquals(state3, handle3.getState(getClass().getClassLoader()));
+			handle3.discardState();
+			
+			assertTrue(isDirectoryEmpty(checkpointDir));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			deleteDirectorySilently(tempDir);
+		}
+	}
+
+	@Test
+	public void testStateOutputStream() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+		try {
+			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+			backend.initializeForJob(new JobID());
+
+			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+			byte[] state1 = new byte[1274673];
+			byte[] state2 = new byte[1];
+			byte[] state3 = new byte[0];
+			byte[] state4 = new byte[177];
+			
+			Random rnd = new Random();
+			rnd.nextBytes(state1);
+			rnd.nextBytes(state2);
+			rnd.nextBytes(state3);
+			rnd.nextBytes(state4);
+
+			long checkpointId = 97231523452L;
+
+			FsStateBackend.FsCheckpointStateOutputStream stream1 = 
+					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+			FsStateBackend.FsCheckpointStateOutputStream stream2 =
+					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+			FsStateBackend.FsCheckpointStateOutputStream stream3 =
+					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+			
+			stream1.write(state1);
+			stream2.write(state2);
+			stream3.write(state3);
+			
+			FileStreamStateHandle handle1 = stream1.closeAndGetHandle();
+			FileStreamStateHandle handle2 = stream2.closeAndGetHandle();
+			FileStreamStateHandle handle3 = stream3.closeAndGetHandle();
+			
+			// use with try-with-resources
+			StreamStateHandle handle4;
+			try (StateBackend.CheckpointStateOutputStream stream4 =
+					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis())) {
+				stream4.write(state4);
+				handle4 = stream4.closeAndGetHandle();
+			}
+			
+			// close before accessing handle
+			StateBackend.CheckpointStateOutputStream stream5 =
+					backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+			stream5.write(state4);
+			stream5.close();
+			try {
+				stream5.closeAndGetHandle();
+				fail();
+			} catch (IOException e) {
+				// uh-huh
+			}
+			
+			validateBytesInStream(handle1.getState(getClass().getClassLoader()), state1);
+			handle1.discardState();
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			ensureLocalFileDeleted(handle1.getFilePath());
+			
+			validateBytesInStream(handle2.getState(getClass().getClassLoader()), state2);
+			handle2.discardState();
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			ensureLocalFileDeleted(handle2.getFilePath());
+			
+			validateBytesInStream(handle3.getState(getClass().getClassLoader()), state3);
+			handle3.discardState();
+			assertFalse(isDirectoryEmpty(checkpointDir));
+			ensureLocalFileDeleted(handle3.getFilePath());
+			
+			validateBytesInStream(handle4.getState(getClass().getClassLoader()), state4);
+			handle4.discardState();
+			assertTrue(isDirectoryEmpty(checkpointDir));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			deleteDirectorySilently(tempDir);
+		}
+	}
+
+	@Test
+	public void testKeyValueState() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+		try {
+			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+			backend.initializeForJob(new JobID());
+
+			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+			KvState<Integer, String, FsStateBackend> kv =
+					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+			assertEquals(0, kv.size());
+
+			// some modifications to the state
+			kv.setCurrentKey(1);
+			assertNull(kv.value());
+			kv.update("1");
+			assertEquals(1, kv.size());
+			kv.setCurrentKey(2);
+			assertNull(kv.value());
+			kv.update("2");
+			assertEquals(2, kv.size());
+			kv.setCurrentKey(1);
+			assertEquals("1", kv.value());
+			assertEquals(2, kv.size());
+
+			// draw a snapshot
+			KvStateSnapshot<Integer, String, FsStateBackend> snapshot1 =
+					kv.shapshot(682375462378L, System.currentTimeMillis());
+
+			// make some more modifications
+			kv.setCurrentKey(1);
+			kv.update("u1");
+			kv.setCurrentKey(2);
+			kv.update("u2");
+			kv.setCurrentKey(3);
+			kv.update("u3");
+
+			// draw another snapshot
+			KvStateSnapshot<Integer, String, FsStateBackend> snapshot2 =
+					kv.shapshot(682375462379L, System.currentTimeMillis());
+
+			// validate the original state
+			assertEquals(3, kv.size());
+			kv.setCurrentKey(1);
+			assertEquals("u1", kv.value());
+			kv.setCurrentKey(2);
+			assertEquals("u2", kv.value());
+			kv.setCurrentKey(3);
+			assertEquals("u3", kv.value());
+
+			// restore the first snapshot and validate it
+			KvState<Integer, String, FsStateBackend> restored1 = snapshot1.restoreState(backend,
+					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+			assertEquals(2, restored1.size());
+			restored1.setCurrentKey(1);
+			assertEquals("1", restored1.value());
+			restored1.setCurrentKey(2);
+			assertEquals("2", restored1.value());
+
+			// restore the first snapshot and validate it
+			KvState<Integer, String, FsStateBackend> restored2 = snapshot2.restoreState(backend,
+					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+			assertEquals(3, restored2.size());
+			restored2.setCurrentKey(1);
+			assertEquals("u1", restored2.value());
+			restored2.setCurrentKey(2);
+			assertEquals("u2", restored2.value());
+			restored2.setCurrentKey(3);
+			assertEquals("u3", restored2.value());
+
+			snapshot1.discardState();
+			assertFalse(isDirectoryEmpty(checkpointDir));
+
+			snapshot2.discardState();
+			assertTrue(isDirectoryEmpty(checkpointDir));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			deleteDirectorySilently(tempDir);
+		}
+	}
+
+	@Test
+	public void testRestoreWithWrongSerializers() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+		try {
+			FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+			backend.initializeForJob(new JobID());
+
+			File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+			
+			KvState<Integer, String, FsStateBackend> kv =
+					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+			kv.setCurrentKey(1);
+			kv.update("1");
+			kv.setCurrentKey(2);
+			kv.update("2");
+
+			KvStateSnapshot<Integer, String, FsStateBackend> snapshot =
+					kv.shapshot(682375462378L, System.currentTimeMillis());
+
+
+			@SuppressWarnings("unchecked")
+			TypeSerializer<Integer> fakeIntSerializer =
+					(TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
+
+			@SuppressWarnings("unchecked")
+			TypeSerializer<String> fakeStringSerializer =
+					(TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
+
+			try {
+				snapshot.restoreState(backend, fakeIntSerializer,
+						StringSerializer.INSTANCE, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+
+			try {
+				snapshot.restoreState(backend, IntSerializer.INSTANCE,
+						fakeStringSerializer, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+
+			try {
+				snapshot.restoreState(backend, fakeIntSerializer,
+						fakeStringSerializer, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+			
+			snapshot.discardState();
+
+			assertTrue(isDirectoryEmpty(checkpointDir));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			deleteDirectorySilently(tempDir);
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+	
+	private static void ensureLocalFileDeleted(Path path) {
+		URI uri = path.toUri();
+		if ("file".equals(uri.getScheme())) {
+			File file = new File(uri.getPath());
+			assertFalse("file not properly deleted", file.exists());
+		}
+		else {
+			throw new IllegalArgumentException("not a local path");
+		}
+	}
+	
+	private static void deleteDirectorySilently(File dir) {
+		try {
+			FileUtils.deleteDirectory(dir);
+		}
+		catch (IOException ignored) {}
+	}
+	
+	private static boolean isDirectoryEmpty(File directory) {
+		String[] nested = directory.list();
+		return  nested == null || nested.length == 0;
+	}
+	
+	private static String localFileUri(File path) {
+		return (OperatingSystem.isWindows() ? "file:/" : "file://") + path.getAbsolutePath();
+	}
+	
+	private static void validateBytesInStream(InputStream is, byte[] data) throws IOException {
+		byte[] holder = new byte[data.length];
+		assertEquals("not enough data", holder.length, is.read(holder));
+		assertEquals("too much data", -1, is.read());
+		assertArrayEquals("wrong data", data, holder);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
new file mode 100644
index 0000000..5f95b33
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
@@ -0,0 +1,282 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
+import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.KvStateSnapshot;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.types.StringValue;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the {@link org.apache.flink.runtime.state.memory.MemoryStateBackend}.
+ */
+public class MemoryStateBackendTest {
+	
+	@Test
+	public void testSerializableState() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend();
+
+			HashMap<String, Integer> state = new HashMap<>();
+			state.put("hey there", 2);
+			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+			
+			StateHandle<HashMap<String, Integer>> handle = backend.checkpointStateSerializable(state, 12, 459);
+			assertNotNull(handle);
+			
+			HashMap<String, Integer> restored = handle.getState(getClass().getClassLoader());
+			assertEquals(state, restored);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testOversizedState() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend(10);
+
+			HashMap<String, Integer> state = new HashMap<>();
+			state.put("hey there", 2);
+			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+			try {
+				backend.checkpointStateSerializable(state, 12, 459);
+				fail("this should cause an exception");
+			}
+			catch (IOException e) {
+				// now darling, isn't that exactly what we wanted?
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testStateStream() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend();
+
+			HashMap<String, Integer> state = new HashMap<>();
+			state.put("hey there", 2);
+			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+			StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
+			ObjectOutputStream oos = new ObjectOutputStream(os);
+			oos.writeObject(state);
+			oos.flush();
+			StreamStateHandle handle = os.closeAndGetHandle();
+			
+			assertNotNull(handle);
+
+			ObjectInputStream ois = new ObjectInputStream(handle.getState(getClass().getClassLoader()));
+			assertEquals(state, ois.readObject());
+			assertTrue(ois.available() <= 0);
+			ois.close();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testOversizedStateStream() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend(10);
+
+			HashMap<String, Integer> state = new HashMap<>();
+			state.put("hey there", 2);
+			state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+			StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
+			ObjectOutputStream oos = new ObjectOutputStream(os);
+			
+			try {
+				oos.writeObject(state);
+				oos.flush();
+				os.closeAndGetHandle();
+				fail("this should cause an exception");
+			}
+			catch (IOException e) {
+				// oh boy! what an exception!
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testKeyValueState() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend();
+			
+			KvState<Integer, String, MemoryStateBackend> kv = 
+					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+			
+			assertEquals(0, kv.size());
+			
+			// some modifications to the state
+			kv.setCurrentKey(1);
+			assertNull(kv.value());
+			kv.update("1");
+			assertEquals(1, kv.size());
+			kv.setCurrentKey(2);
+			assertNull(kv.value());
+			kv.update("2");
+			assertEquals(2, kv.size());
+			kv.setCurrentKey(1);
+			assertEquals("1", kv.value());
+			assertEquals(2, kv.size());
+			
+			// draw a snapshot
+			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot1 = 
+					kv.shapshot(682375462378L, System.currentTimeMillis());
+			
+			// make some more modifications
+			kv.setCurrentKey(1);
+			kv.update("u1");
+			kv.setCurrentKey(2);
+			kv.update("u2");
+			kv.setCurrentKey(3);
+			kv.update("u3");
+
+			// draw another snapshot
+			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot2 =
+					kv.shapshot(682375462379L, System.currentTimeMillis());
+			
+			// validate the original state
+			assertEquals(3, kv.size());
+			kv.setCurrentKey(1);
+			assertEquals("u1", kv.value());
+			kv.setCurrentKey(2);
+			assertEquals("u2", kv.value());
+			kv.setCurrentKey(3);
+			assertEquals("u3", kv.value());
+			
+			// restore the first snapshot and validate it
+			KvState<Integer, String, MemoryStateBackend> restored1 = snapshot1.restoreState(backend, 
+							IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+			assertEquals(2, restored1.size());
+			restored1.setCurrentKey(1);
+			assertEquals("1", restored1.value());
+			restored1.setCurrentKey(2);
+			assertEquals("2", restored1.value());
+
+			// restore the first snapshot and validate it
+			KvState<Integer, String, MemoryStateBackend> restored2 = snapshot2.restoreState(backend,
+					IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+			assertEquals(3, restored2.size());
+			restored2.setCurrentKey(1);
+			assertEquals("u1", restored2.value());
+			restored2.setCurrentKey(2);
+			assertEquals("u2", restored2.value());
+			restored2.setCurrentKey(3);
+			assertEquals("u3", restored2.value());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRestoreWithWrongSerializers() {
+		try {
+			MemoryStateBackend backend = new MemoryStateBackend();
+			KvState<Integer, String, MemoryStateBackend> kv =
+					backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+			
+			kv.setCurrentKey(1);
+			kv.update("1");
+			kv.setCurrentKey(2);
+			kv.update("2");
+			
+			KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot =
+					kv.shapshot(682375462378L, System.currentTimeMillis());
+
+
+			@SuppressWarnings("unchecked")
+			TypeSerializer<Integer> fakeIntSerializer = 
+					(TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
+
+			@SuppressWarnings("unchecked")
+			TypeSerializer<String> fakeStringSerializer = 
+					(TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
+
+			try {
+				snapshot.restoreState(backend, fakeIntSerializer,
+						StringSerializer.INSTANCE, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+
+			try {
+				snapshot.restoreState(backend, IntSerializer.INSTANCE,
+						fakeStringSerializer, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+
+			try {
+				snapshot.restoreState(backend, fakeIntSerializer,
+						fakeStringSerializer, null, getClass().getClassLoader());
+				fail("should recognize wrong serializers");
+			} catch (IllegalArgumentException e) {
+				// expected
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
index d2e5b6a..a65ec01 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.testutils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
@@ -79,7 +80,7 @@ public class ZooKeeperTestUtils {
 
 		// File system state backend
 		config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-		config.setString(ConfigConstants.STATE_BACKEND_FS_DIR, fsStateHandlePath + "/checkpoints");
+		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, fsStateHandlePath + "/checkpoints");
 		config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, fsStateHandlePath + "/recovery");
 
 		// Akka failure detection and execution retries

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
index f0130ec..788f70d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
@@ -23,7 +23,6 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.TestLogger;
 import org.apache.zookeeper.CreateMode;
@@ -83,11 +82,9 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	 */
 	@Test
 	public void testAdd() throws Exception {
-		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
-
-		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
-				ZooKeeper.getClient(), stateHandleProvider);
+		LongStateStorage longStateStorage = new LongStateStorage();
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<Long>(
+				ZooKeeper.getClient(), longStateStorage);
 
 		// Config
 		final String pathInZooKeeper = "/testAdd";
@@ -98,8 +95,8 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 
 		// Verify
 		// State handle created
-		assertEquals(1, stateHandleProvider.getStateHandles().size());
-		assertEquals(state, stateHandleProvider.getStateHandles().get(0).getState(null));
+		assertEquals(1, store.getAll().size());
+		assertEquals(state, store.get(pathInZooKeeper).getState(null));
 
 		// Path created and is persistent
 		Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper);
@@ -120,10 +117,9 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	 */
 	@Test
 	public void testAddWithCreateMode() throws Exception {
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
-
-		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
-				ZooKeeper.getClient(), stateHandleProvider);
+		LongStateStorage longStateStorage = new LongStateStorage();
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<Long>(
+				ZooKeeper.getClient(), longStateStorage);
 
 		// Config
 		Long state = 3457347234L;
@@ -151,8 +147,8 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 
 			// Verify
 			// State handle created
-			assertEquals(i + 1, stateHandleProvider.getStateHandles().size());
-			assertEquals(state, stateHandleProvider.getStateHandles().get(i).getState(null));
+			assertEquals(i + 1, store.getAll().size());
+			assertEquals(state, longStateStorage.getStateHandles().get(i).getState(null));
 
 			// Path created
 			Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper);
@@ -182,7 +178,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	 */
 	@Test(expected = Exception.class)
 	public void testAddAlreadyExistingPath() throws Exception {
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -198,7 +194,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testAddDiscardStateHandleAfterFailure() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		CuratorFramework client = spy(ZooKeeper.getClient());
 		when(client.create()).thenThrow(new RuntimeException("Expected test Exception."));
@@ -231,7 +227,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testReplace() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -270,10 +266,10 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	 */
 	@Test(expected = Exception.class)
 	public void testReplaceNonExistingPath() throws Exception {
-		StateHandleProvider<Long> stateHandleProvider = new LongStateHandleProvider();
+		StateStorageHelper<Long> stateStorage = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
-				ZooKeeper.getClient(), stateHandleProvider);
+				ZooKeeper.getClient(), stateStorage);
 
 		store.replace("/testReplaceNonExistingPath", 0, 1L);
 	}
@@ -284,7 +280,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testReplaceDiscardStateHandleAfterFailure() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		CuratorFramework client = spy(ZooKeeper.getClient());
 		when(client.setData()).thenThrow(new RuntimeException("Expected test Exception."));
@@ -329,7 +325,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testGetAndExists() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -354,7 +350,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	 */
 	@Test(expected = Exception.class)
 	public void testGetNonExistingPath() throws Exception {
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -368,7 +364,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testGetAll() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -399,7 +395,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testGetAllSortedByName() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -429,7 +425,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testRemove() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -453,7 +449,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testRemoveWithCallback() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -492,7 +488,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testRemoveAndDiscardState() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -514,7 +510,7 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	@Test
 	public void testRemoveAndDiscardAllState() throws Exception {
 		// Setup
-		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+		LongStateStorage stateHandleProvider = new LongStateStorage();
 
 		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
 				ZooKeeper.getClient(), stateHandleProvider);
@@ -543,21 +539,19 @@ public class ZooKeeperStateHandleStoreITCase extends TestLogger {
 	// Simple test helpers
 	// ---------------------------------------------------------------------------------------------
 
-	private static class LongStateHandleProvider implements StateHandleProvider<Long> {
-
-		private static final long serialVersionUID = 4572084854499402276L;
+	private static class LongStateStorage implements StateStorageHelper<Long> {
 
 		private final List<LongStateHandle> stateHandles = new ArrayList<>();
 
 		@Override
-		public StateHandle<Long> createStateHandle(Long state) {
+		public StateHandle<Long> store(Long state) throws Exception {
 			LongStateHandle stateHandle = new LongStateHandle(state);
 			stateHandles.add(stateHandle);
 
 			return stateHandle;
 		}
 
-		public List<LongStateHandle> getStateHandles() {
+		List<LongStateHandle> getStateHandles() {
 			return stateHandles;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties
index 1ca02aa..f77ed07 100644
--- a/flink-runtime/src/test/resources/log4j-test.properties
+++ b/flink-runtime/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=INFO, console
+log4j.rootLogger=OFF, console
 
 # -----------------------------------------------------------------------------
 # Console (use 'console')
@@ -36,3 +36,4 @@ log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
 log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console
+log4j.logger.org.apache.flink.runtime.blob=DEBUG

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
index 8b7fb1c..4e4acd2 100644
--- a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
+++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
@@ -27,11 +27,11 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.filesystem.FileStreamStateHandle;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.filesystem.FileStreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.StreamStateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 26e1c9e..98506e0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -63,7 +63,7 @@ import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
 import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.api.transformations.StreamTransformation;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.SplittableIterator;
@@ -372,11 +372,11 @@ public abstract class StreamExecutionEnvironment {
 	 * the key/value state, and for checkpointed functions (implementing the interface
 	 * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed}).
 	 *
-	 * <p>The {@link org.apache.flink.streaming.api.state.memory.MemoryStateBackend} for example
+	 * <p>The {@link org.apache.flink.runtime.state.memory.MemoryStateBackend} for example
 	 * maintains the state in heap memory, as objects. It is lightweight without extra dependencies,
 	 * but can checkpoint only small states (some counters).
 	 * 
-	 * <p>In contrast, the {@link org.apache.flink.streaming.api.state.filesystem.FsStateBackend}
+	 * <p>In contrast, the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend}
 	 * stores checkpoints of the state (also maintained as heap objects) in files. When using a replicated
 	 * file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee that state is not lost upon
 	 * failures of individual nodes and that streaming program can be executed highly available and strongly

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
index 3817ede..3ac63af 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
@@ -26,8 +26,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.state.SerializedCheckpointData;
+import org.apache.flink.runtime.state.SerializedCheckpointData;
 
 import java.util.ArrayDeque;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
index 76be598..11bf84f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.util.ClassLoaderUtil;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
 import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.runtime.tasks.StreamTaskException;
 import org.apache.flink.util.InstantiationUtil;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index 0652406..be020d7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -50,7 +50,7 @@ import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamSource;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index 9e60e9a..078679d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -24,9 +24,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.state.KvState;
-import org.apache.flink.streaming.api.state.KvStateSnapshot;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.KvStateSnapshot;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
index a991fd3..17bd08d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
@@ -27,7 +27,7 @@ import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import org.apache.flink.streaming.runtime.tasks.StreamTaskState;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
deleted file mode 100644
index b974674..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Base class for key/value state implementations that are backed by a regular heap hash map. The
- * concrete implementations define how the state is checkpointed.
- * 
- * @param <K> The type of the key.
- * @param <V> The type of the value.
- * @param <Backend> The type of the backend that snapshots this key/value state.
- */
-public abstract class AbstractHeapKvState<K, V, Backend extends StateBackend<Backend>> implements KvState<K, V, Backend> {
-
-	/** Map containing the actual key/value pairs */
-	private final HashMap<K, V> state;
-	
-	/** The serializer for the keys */
-	private final TypeSerializer<K> keySerializer;
-
-	/** The serializer for the values */
-	private final TypeSerializer<V> valueSerializer;
-	
-	/** The value that is returned when no other value has been associated with a key, yet */
-	private final V defaultValue;
-	
-	/** The current key, which the next value methods will refer to */
-	private K currentKey;
-	
-	/**
-	 * Creates a new empty key/value state.
-	 * 
-	 * @param keySerializer The serializer for the keys.
-	 * @param valueSerializer The serializer for the values.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
-	 */
-	protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
-									TypeSerializer<V> valueSerializer,
-									V defaultValue) {
-		this(keySerializer, valueSerializer, defaultValue, new HashMap<K, V>());
-	}
-
-	/**
-	 * Creates a new key/value state for the given hash map of key/value pairs.
-	 * 
-	 * @param keySerializer The serializer for the keys.
-	 * @param valueSerializer The serializer for the values.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
-	 * @param state The state map to use in this kev/value state. May contain initial state.   
-	 */
-	protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
-									TypeSerializer<V> valueSerializer,
-									V defaultValue,
-									HashMap<K, V> state) {
-		this.state = requireNonNull(state);
-		this.keySerializer = requireNonNull(keySerializer);
-		this.valueSerializer = requireNonNull(valueSerializer);
-		this.defaultValue = defaultValue;
-	}
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public V value() {
-		V value = state.get(currentKey);
-		return value != null ? value : defaultValue;
-	}
-
-	@Override
-	public void update(V value) {
-		if (value != null) {
-			state.put(currentKey, value);
-		}
-		else {
-			state.remove(currentKey);
-		}
-	}
-
-	@Override
-	public void setCurrentKey(K currentKey) {
-		this.currentKey = currentKey;
-	}
-
-	@Override
-	public int size() {
-		return state.size();
-	}
-
-	@Override
-	public void dispose() {
-		state.clear();
-	}
-
-	/**
-	 * Gets the serializer for the keys.
-	 * @return The serializer for the keys.
-	 */
-	public TypeSerializer<K> getKeySerializer() {
-		return keySerializer;
-	}
-
-	/**
-	 * Gets the serializer for the values.
-	 * @return The serializer for the values.
-	 */
-	public TypeSerializer<V> getValueSerializer() {
-		return valueSerializer;
-	}
-
-	// ------------------------------------------------------------------------
-	//  checkpointing utilities
-	// ------------------------------------------------------------------------
-	
-	protected void writeStateToOutputView(final DataOutputView out) throws IOException {
-		for (Map.Entry<K, V> entry : state.entrySet()) {
-			keySerializer.serialize(entry.getKey(), out);
-			valueSerializer.serialize(entry.getValue(), out);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
deleted file mode 100644
index 9c628f8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.state.OperatorState;
-
-/**
- * Key/Value state implementation for user-defined state. The state is backed by a state
- * backend, which typically follows one of the following patterns: Either the state is stored
- * in the key/value state object directly (meaning in the executing JVM) and snapshotted by the
- * state backend into some store (during checkpoints), or the key/value state is in fact backed
- * by an external key/value store as the state backend, and checkpoints merely record the
- * metadata of what is considered part of the checkpoint.
- * 
- * @param <K> The type of the key.
- * @param <V> The type of the value.
- */
-public interface KvState<K, V, Backend extends StateBackend<Backend>> extends OperatorState<V> {
-
-	/**
-	 * Sets the current key, which will be used to retrieve values for the next calls to
-	 * {@link #value()} and {@link #update(Object)}.
-	 * 
-	 * @param key The key.
-	 */
-	void setCurrentKey(K key);
-
-	/**
-	 * Creates a snapshot of this state.
-	 * 
-	 * @param checkpointId The ID of the checkpoint for which the snapshot should be created.
-	 * @param timestamp The timestamp of the checkpoint.
-	 * @return A snapshot handle for this key/value state.
-	 * 
-	 * @throws Exception Exceptions during snapshotting the state should be forwarded, so the system
-	 *                   can react to failed snapshots.
-	 */
-	KvStateSnapshot<K, V, Backend> shapshot(long checkpointId, long timestamp) throws Exception;
-
-	/**
-	 * Gets the number of key/value pairs currently stored in the state. Note that is a key
-	 * has been associated with "null", the key is removed from the state an will not
-	 * be counted here.
-	 *
-	 * @return The number of key/value pairs currently stored in the state.
-	 */
-	int size();
-
-	/**
-	 * Disposes the key/value state, releasing all occupied resources.
-	 */
-	void dispose();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
deleted file mode 100644
index 6aa7a1e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * This class represents a snapshot of the {@link KvState}, taken for a checkpoint. Where exactly
- * the snapshot stores the snapshot data (in this object, in an external data store, etc) depends
- * on the actual implementation. This snapshot defines merely how to restore the state and
- * how to discard the state.
- *
- * <p>One possible implementation is that this snapshot simply contains a copy of the key/value map.
- * 
- * <p>Another possible implementation for this snapshot is that the key/value map is serialized into
- * a file and this snapshot object contains a pointer to that file.
- *
- * @param <K> The type of the key
- * @param <V> The type of the value
- * @param <Backend> The type of the backend that can restore the state from this snapshot.
- */
-public interface KvStateSnapshot<K, V, Backend extends StateBackend<Backend>> extends java.io.Serializable {
-
-	/**
-	 * Loads the key/value state back from this snapshot.
-	 * 
-	 * 
-	 * @param stateBackend The state backend that created this snapshot and can restore the key/value state
-	 *                     from this snapshot.
-	 * @param keySerializer The serializer for the keys.
-	 * @param valueSerializer The serializer for the values.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.   
-	 * @param classLoader The class loader for user-defined types.
-	 * 
-	 * @return An instance of the key/value state loaded from this snapshot.
-	 * 
-	 * @throws Exception Exceptions can occur during the state loading and are forwarded. 
-	 */
-	KvState<K, V, Backend> restoreState(
-			Backend stateBackend,
-			TypeSerializer<K> keySerializer,
-			TypeSerializer<V> valueSerializer,
-			V defaultValue,
-			ClassLoader classLoader) throws Exception;
-
-
-	/**
-	 * Discards the state snapshot, removing any resources occupied by it.
-	 * 
-	 * @throws Exception Exceptions occurring during the state disposal should be forwarded.
-	 */
-	void discardState() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/SerializedCheckpointData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/SerializedCheckpointData.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/SerializedCheckpointData.java
deleted file mode 100644
index 2bbb4e2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/SerializedCheckpointData.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * This class represents serialized checkpoint data for a collection of elements.
- */
-public class SerializedCheckpointData implements java.io.Serializable {
-
-	private static final long serialVersionUID = -8783744683896503488L;
-	
-	/** ID of the checkpoint for which the IDs are stored */
-	private final long checkpointId;
-
-	/** The serialized elements */
-	private final byte[] serializedData;
-
-	/** The number of elements in the checkpoint */
-	private final int numIds;
-
-	/**
-	 * Creates a SerializedCheckpointData object for the given serialized data.
-	 * 
-	 * @param checkpointId The checkpointId of the checkpoint.
-	 * @param serializedData The serialized IDs in this checkpoint.
-	 * @param numIds The number of IDs in the checkpoint.
-	 */
-	public SerializedCheckpointData(long checkpointId, byte[] serializedData, int numIds) {
-		this.checkpointId = checkpointId;
-		this.serializedData = serializedData;
-		this.numIds = numIds;
-	}
-
-	/**
-	 * Gets the checkpointId of the checkpoint.
-	 * @return The checkpointId of the checkpoint.
-	 */
-	public long getCheckpointId() {
-		return checkpointId;
-	}
-
-	/**
-	 * Gets the binary data for the serialized elements.
-	 * @return The binary data for the serialized elements.
-	 */
-	public byte[] getSerializedData() {
-		return serializedData;
-	}
-
-	/**
-	 * Gets the number of IDs in the checkpoint.
-	 * @return The number of IDs in the checkpoint.
-	 */
-	public int getNumIds() {
-		return numIds;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Serialize to Checkpoint
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Converts a list of checkpoints with elements into an array of SerializedCheckpointData.
-	 * 
-	 * @param checkpoints The checkpoints to be converted into IdsCheckpointData.
-	 * @param serializer The serializer to serialize the IDs.
-	 * @param <T> The type of the ID.
-	 * @return An array of serializable SerializedCheckpointData, one per entry in the 
-	 * 
-	 * @throws IOException Thrown, if the serialization fails.
-	 */
-	public static <T> SerializedCheckpointData[] fromDeque(ArrayDeque<Tuple2<Long, List<T>>> checkpoints,
-												TypeSerializer<T> serializer) throws IOException {
-		return fromDeque(checkpoints, serializer, new DataOutputSerializer(128));
-	}
-
-	/**
-	 * Converts a list of checkpoints into an array of SerializedCheckpointData.
-	 *
-	 * @param checkpoints The checkpoints to be converted into IdsCheckpointData.
-	 * @param serializer The serializer to serialize the IDs.
-	 * @param outputBuffer The reusable serialization buffer.
-	 * @param <T> The type of the ID.
-	 * @return An array of serializable SerializedCheckpointData, one per entry in the 
-	 *
-	 * @throws IOException Thrown, if the serialization fails.
-	 */
-	public static <T> SerializedCheckpointData[] fromDeque(ArrayDeque<Tuple2<Long, List<T>>> checkpoints,
-												TypeSerializer<T> serializer,
-												DataOutputSerializer outputBuffer) throws IOException {
-		SerializedCheckpointData[] serializedCheckpoints = new SerializedCheckpointData[checkpoints.size()];
-		
-		int pos = 0;
-		for (Tuple2<Long, List<T>> checkpoint : checkpoints) {
-			outputBuffer.clear();
-			List<T> checkpointIds = checkpoint.f1;
-			
-			for (T id : checkpointIds) {
-				serializer.serialize(id, outputBuffer);
-			}
-
-			serializedCheckpoints[pos++] = new SerializedCheckpointData(
-					checkpoint.f0, outputBuffer.getCopyOfBuffer(), checkpointIds.size());
-		}
-		
-		return serializedCheckpoints;
-	}
-
-	// ------------------------------------------------------------------------
-	//  De-Serialize from Checkpoint
-	// ------------------------------------------------------------------------
-
-	/**
-	 * De-serializes an array of SerializedCheckpointData back into an ArrayDeque of element checkpoints.
-	 * 
-	 * @param data The data to be deserialized.
-	 * @param serializer The serializer used to deserialize the data.
-	 * @param <T> The type of the elements.
-	 * @return An ArrayDeque of element checkpoints.
-	 * 
-	 * @throws IOException Thrown, if the serialization fails.
-	 */
-	public static <T> ArrayDeque<Tuple2<Long, List<T>>> toDeque(
-			SerializedCheckpointData[] data, TypeSerializer<T> serializer) throws IOException
-	{
-		ArrayDeque<Tuple2<Long, List<T>>> deque = new ArrayDeque<>(data.length);
-		DataInputDeserializer deser = null;
-		
-		for (SerializedCheckpointData checkpoint : data) {
-			byte[] serializedData = checkpoint.getSerializedData();
-			if (deser == null) {
-				deser = new DataInputDeserializer(serializedData, 0, serializedData.length);
-			}
-			else {
-				deser.setBuffer(serializedData, 0, serializedData.length);
-			}
-			
-			final List<T> ids = new ArrayList<>(checkpoint.getNumIds());
-			final int numIds = checkpoint.getNumIds();
-			
-			for (int i = 0; i < numIds; i++) {
-				ids.add(serializer.deserialize(deser));
-			}
-
-			deque.addLast(new Tuple2<Long, List<T>>(checkpoint.checkpointId, ids));
-		}
-		
-		return deque;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
deleted file mode 100644
index f4391ad..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.state.StateHandle;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-/**
- * A state backend defines how state is stored and snapshotted during checkpoints.
- * 
- * @param <Backend> The type of backend itself. This generic parameter is used to refer to the
- *                  type of backend when creating state backed by this backend.
- */
-public abstract class StateBackend<Backend extends StateBackend<Backend>> implements java.io.Serializable {
-	
-	private static final long serialVersionUID = 4620413814639220247L;
-	
-	// ------------------------------------------------------------------------
-	//  initialization and cleanup
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * This method is called by the task upon deployment to initialize the state backend for
-	 * data for a specific job.
-	 * 
-	 * @param job The ID of the job for which the state backend instance checkpoints data.
-	 * @throws Exception Overwritten versions of this method may throw exceptions, in which
-	 *                   case the job that uses the state backend is considered failed during
-	 *                   deployment.
-	 */
-	public abstract void initializeForJob(JobID job) throws Exception;
-
-	/**
-	 * Disposes all state associated with the current job.
-	 * 
-	 * @throws Exception Exceptions may occur during disposal of the state and should be forwarded.
-	 */
-	public abstract void disposeAllStateForCurrentJob() throws Exception;
-
-	/**
-	 * Closes the state backend, releasing all internal resources, but does not delete any persistent
-	 * checkpoint data.
-	 * 
-	 * @throws Exception Exceptions can be forwarded and will be logged by the system
-	 */
-	public abstract void close() throws Exception;
-	
-	// ------------------------------------------------------------------------
-	//  key/value state
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a key/value state backed by this state backend.
-	 * 
-	 * @param keySerializer The serializer for the key.
-	 * @param valueSerializer The serializer for the value.
-	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
-	 * @param <K> The type of the key.
-	 * @param <V> The type of the value.
-	 * 
-	 * @return A new key/value state backed by this backend.
-	 * 
-	 * @throws Exception Exceptions may occur during initialization of the state and should be forwarded.
-	 */
-	public abstract <K, V> KvState<K, V, Backend> createKvState(
-			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
-			V defaultValue) throws Exception;
-	
-	
-	// ------------------------------------------------------------------------
-	//  storing state for a checkpoint
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates an output stream that writes into the state of the given checkpoint. When the stream
-	 * is closes, it returns a state handle that can retrieve the state back.
-	 * 
-	 * @param checkpointID The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 * @return An output stream that writes state for the given checkpoint.
-	 * 
-	 * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
-	 */
-	public abstract CheckpointStateOutputStream createCheckpointStateOutputStream(
-			long checkpointID, long timestamp) throws Exception;
-	
-	/**
-	 * Creates a {@link DataOutputView} stream that writes into the state of the given checkpoint.
-	 * When the stream is closes, it returns a state handle that can retrieve the state back.
-	 *
-	 * @param checkpointID The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 * @return An DataOutputView stream that writes state for the given checkpoint.
-	 *
-	 * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
-	 */
-	public CheckpointStateOutputView createCheckpointStateOutputView(
-			long checkpointID, long timestamp) throws Exception {
-		return new CheckpointStateOutputView(createCheckpointStateOutputStream(checkpointID, timestamp));
-	}
-
-	/**
-	 * Writes the given state into the checkpoint, and returns a handle that can retrieve the state back.
-	 * 
-	 * @param state The state to be checkpointed.
-	 * @param checkpointID The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 * @param <S> The type of the state.
-	 * 
-	 * @return A state handle that can retrieve the checkpoined state.
-	 * 
-	 * @throws Exception Exceptions may occur during serialization / storing the state and should be forwarded.
-	 */
-	public abstract <S extends Serializable> StateHandle<S> checkpointStateSerializable(
-			S state, long checkpointID, long timestamp) throws Exception;
-	
-	
-	// ------------------------------------------------------------------------
-	//  Checkpoint state output stream
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A dedicated output stream that produces a {@link StreamStateHandle} when closed.
-	 */
-	public static abstract class CheckpointStateOutputStream extends OutputStream {
-
-		/**
-		 * Closes the stream and gets a state handle that can create an input stream
-		 * producing the data written to this stream.
-		 * 
-		 * @return A state handle that can create an input stream producing the data written to this stream.
-		 * @throws IOException Thrown, if the stream cannot be closed.
-		 */
-		public abstract StreamStateHandle closeAndGetHandle() throws IOException;
-	}
-
-	/**
-	 * A dedicated DataOutputView stream that produces a {@code StateHandle<DataInputView>} when closed.
-	 */
-	public static final class CheckpointStateOutputView extends DataOutputViewStreamWrapper {
-		
-		private final CheckpointStateOutputStream out;
-		
-		public CheckpointStateOutputView(CheckpointStateOutputStream out) {
-			super(out);
-			this.out = out;
-		}
-
-		/**
-		 * Closes the stream and gets a state handle that can create a DataInputView.
-		 * producing the data written to this stream.
-		 *
-		 * @return A state handle that can create an input stream producing the data written to this stream.
-		 * @throws IOException Thrown, if the stream cannot be closed.
-		 */
-		public StateHandle<DataInputView> closeAndGetHandle() throws IOException {
-			return new DataInputViewHandle(out.closeAndGetHandle());
-		}
-
-		@Override
-		public void close() throws IOException {
-			out.close();
-		}
-	}
-
-	/**
-	 * Simple state handle that resolved a {@link DataInputView} from a StreamStateHandle.
-	 */
-	private static final class DataInputViewHandle implements StateHandle<DataInputView> {
-
-		private static final long serialVersionUID = 2891559813513532079L;
-		
-		private final StreamStateHandle stream;
-
-		private DataInputViewHandle(StreamStateHandle stream) {
-			this.stream = stream;
-		}
-
-		@Override
-		public DataInputView getState(ClassLoader userCodeClassLoader) throws Exception {
-			return new DataInputViewStreamWrapper(stream.getState(userCodeClassLoader)); 
-		}
-
-		@Override
-		public void discardState() throws Exception {
-			stream.discardState();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
deleted file mode 100644
index ad87eae..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.configuration.Configuration;
-
-/**
- * A factory to create a specific state backend. The state backend creation gets a Configuration
- * object that can be used to read further config values.
- * 
- * @param <T> The type of the state backend created.
- */
-public interface StateBackendFactory<T extends StateBackend<T>> {
-
-	/**
-	 * Creates the state backend, optionally using the given configuration.
-	 * 
-	 * @param config The Flink configuration (loaded by the TaskManager).
-	 * @return The created state backend. 
-	 * 
-	 * @throws Exception Exceptions during instantiation can be forwarded.
-	 */
-	StateBackend<T> createFromConfig(Configuration config) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
deleted file mode 100644
index 0fa5952..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state;
-
-import org.apache.flink.runtime.state.StateHandle;
-
-import java.io.InputStream;
-
-/**
- * A state handle that produces an input stream when resolved.
- */
-public interface StreamStateHandle extends StateHandle<InputStream> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
deleted file mode 100644
index c4a376e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-
-import java.io.IOException;
-
-/**
- * Base class for state that is stored in a file.
- */
-public abstract class AbstractFileState implements java.io.Serializable {
-	
-	private static final long serialVersionUID = 350284443258002355L;
-	
-	/** The path to the file in the filesystem, fully describing the file system */
-	private final Path filePath;
-
-	/** Cached file system handle */
-	private transient FileSystem fs;
-
-	/**
-	 * Creates a new file state for the given file path.
-	 * 
-	 * @param filePath The path to the file that stores the state.
-	 */
-	protected AbstractFileState(Path filePath) {
-		this.filePath = filePath;
-	}
-
-	/**
-	 * Gets the path where this handle's state is stored.
-	 * @return The path where this handle's state is stored.
-	 */
-	public Path getFilePath() {
-		return filePath;
-	}
-
-	/**
-	 * Discard the state by deleting the file that stores the state. If the parent directory
-	 * of the state is empty after deleting the state file, it is also deleted.
-	 * 
-	 * @throws Exception Thrown, if the file deletion (not the directory deletion) fails.
-	 */
-	public void discardState() throws Exception {
-		getFileSystem().delete(filePath, false);
-
-		// send a call to delete the directory containing the file. this will
-		// fail (and be ignored) when some files still exist
-		try {
-			getFileSystem().delete(filePath.getParent(), false);
-		} catch (IOException ignored) {}
-	}
-
-	/**
-	 * Gets the file system that stores the file state.
-	 * @return The file system that stores the file state.
-	 * @throws IOException Thrown if the file system cannot be accessed.
-	 */
-	protected FileSystem getFileSystem() throws IOException {
-		if (fs == null) {
-			fs = FileSystem.get(filePath.toUri());
-		}
-		return fs;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
deleted file mode 100644
index 9bf5ec1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.state.filesystem;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.ObjectInputStream;
-
-/**
- * A state handle that points to state stored in a file via Java Serialization.
- * 
- * @param <T> The type of state pointed to by the state handle.
- */
-public class FileSerializableStateHandle<T> extends AbstractFileState implements StateHandle<T> {
-
-	private static final long serialVersionUID = -657631394290213622L;
-	
-	/**
-	 * Creates a new FileSerializableStateHandle pointing to state at the given file path.
-	 * 
-	 * @param filePath The path to the file containing the checkpointed state.
-	 */
-	public FileSerializableStateHandle(Path filePath) {
-		super(filePath);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public T getState(ClassLoader classLoader) throws Exception {
-		FSDataInputStream inStream = getFileSystem().open(getFilePath());
-		ObjectInputStream ois = new InstantiationUtil.ClassLoaderObjectInputStream(inStream, classLoader);
-		return (T) ois.readObject();
-	}
-}


[46/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/css/vendor.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/css/vendor.css b/flink-runtime-web/src/main/resources/web/css/vendor.css
deleted file mode 100644
index 672e07f..0000000
--- a/flink-runtime-web/src/main/resources/web/css/vendor.css
+++ /dev/null
@@ -1,9183 +0,0 @@
-/*!
- *  Font Awesome 4.3.0 by @davegandy - http://fontawesome.io - @fontawesome
- *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
- */
-/* FONT PATH
- * -------------------------- */
-@font-face {
-  font-family: 'FontAwesome';
-  src: url('../fonts/fontawesome-webfont.eot?v=4.3.0');
-  src: url('../fonts/fontawesome-webfont.eot?#iefix&v=4.3.0') format('embedded-opentype'), url('../fonts/fontawesome-webfont.woff2?v=4.3.0') format('woff2'), url('../fonts/fontawesome-webfont.woff?v=4.3.0') format('woff'), url('../fonts/fontawesome-webfont.ttf?v=4.3.0') format('truetype'), url('../fonts/fontawesome-webfont.svg?v=4.3.0#fontawesomeregular') format('svg');
-  font-weight: normal;
-  font-style: normal;
-}
-.fa {
-  display: inline-block;
-  font: normal normal normal 14px/1 FontAwesome;
-  font-size: inherit;
-  text-rendering: auto;
-  -webkit-font-smoothing: antialiased;
-  -moz-osx-font-smoothing: grayscale;
-  transform: translate(0, 0);
-}
-/* makes the font 33% larger relative to the icon container */
-.fa-lg {
-  font-size: 1.33333333em;
-  line-height: 0.75em;
-  vertical-align: -15%;
-}
-.fa-2x {
-  font-size: 2em;
-}
-.fa-3x {
-  font-size: 3em;
-}
-.fa-4x {
-  font-size: 4em;
-}
-.fa-5x {
-  font-size: 5em;
-}
-.fa-fw {
-  width: 1.28571429em;
-  text-align: center;
-}
-.fa-ul {
-  padding-left: 0;
-  margin-left: 2.14285714em;
-  list-style-type: none;
-}
-.fa-ul > li {
-  position: relative;
-}
-.fa-li {
-  position: absolute;
-  left: -2.14285714em;
-  width: 2.14285714em;
-  top: 0.14285714em;
-  text-align: center;
-}
-.fa-li.fa-lg {
-  left: -1.85714286em;
-}
-.fa-border {
-  padding: .2em .25em .15em;
-  border: solid 0.08em #eeeeee;
-  border-radius: .1em;
-}
-.pull-right {
-  float: right;
-}
-.pull-left {
-  float: left;
-}
-.fa.pull-left {
-  margin-right: .3em;
-}
-.fa.pull-right {
-  margin-left: .3em;
-}
-.fa-spin {
-  -webkit-animation: fa-spin 2s infinite linear;
-  animation: fa-spin 2s infinite linear;
-}
-.fa-pulse {
-  -webkit-animation: fa-spin 1s infinite steps(8);
-  animation: fa-spin 1s infinite steps(8);
-}
-@-webkit-keyframes fa-spin {
-  0% {
-    -webkit-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-  100% {
-    -webkit-transform: rotate(359deg);
-    transform: rotate(359deg);
-  }
-}
-@keyframes fa-spin {
-  0% {
-    -webkit-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-  100% {
-    -webkit-transform: rotate(359deg);
-    transform: rotate(359deg);
-  }
-}
-.fa-rotate-90 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=1);
-  -webkit-transform: rotate(90deg);
-  -ms-transform: rotate(90deg);
-  transform: rotate(90deg);
-}
-.fa-rotate-180 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2);
-  -webkit-transform: rotate(180deg);
-  -ms-transform: rotate(180deg);
-  transform: rotate(180deg);
-}
-.fa-rotate-270 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=3);
-  -webkit-transform: rotate(270deg);
-  -ms-transform: rotate(270deg);
-  transform: rotate(270deg);
-}
-.fa-flip-horizontal {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);
-  -webkit-transform: scale(-1, 1);
-  -ms-transform: scale(-1, 1);
-  transform: scale(-1, 1);
-}
-.fa-flip-vertical {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);
-  -webkit-transform: scale(1, -1);
-  -ms-transform: scale(1, -1);
-  transform: scale(1, -1);
-}
-:root .fa-rotate-90,
-:root .fa-rotate-180,
-:root .fa-rotate-270,
-:root .fa-flip-horizontal,
-:root .fa-flip-vertical {
-  filter: none;
-}
-.fa-stack {
-  position: relative;
-  display: inline-block;
-  width: 2em;
-  height: 2em;
-  line-height: 2em;
-  vertical-align: middle;
-}
-.fa-stack-1x,
-.fa-stack-2x {
-  position: absolute;
-  left: 0;
-  width: 100%;
-  text-align: center;
-}
-.fa-stack-1x {
-  line-height: inherit;
-}
-.fa-stack-2x {
-  font-size: 2em;
-}
-.fa-inverse {
-  color: #ffffff;
-}
-/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen
-   readers do not read off random characters that represent icons */
-.fa-glass:before {
-  content: "\f000";
-}
-.fa-music:before {
-  content: "\f001";
-}
-.fa-search:before {
-  content: "\f002";
-}
-.fa-envelope-o:before {
-  content: "\f003";
-}
-.fa-heart:before {
-  content: "\f004";
-}
-.fa-star:before {
-  content: "\f005";
-}
-.fa-star-o:before {
-  content: "\f006";
-}
-.fa-user:before {
-  content: "\f007";
-}
-.fa-film:before {
-  content: "\f008";
-}
-.fa-th-large:before {
-  content: "\f009";
-}
-.fa-th:before {
-  content: "\f00a";
-}
-.fa-th-list:before {
-  content: "\f00b";
-}
-.fa-check:before {
-  content: "\f00c";
-}
-.fa-remove:before,
-.fa-close:before,
-.fa-times:before {
-  content: "\f00d";
-}
-.fa-search-plus:before {
-  content: "\f00e";
-}
-.fa-search-minus:before {
-  content: "\f010";
-}
-.fa-power-off:before {
-  content: "\f011";
-}
-.fa-signal:before {
-  content: "\f012";
-}
-.fa-gear:before,
-.fa-cog:before {
-  content: "\f013";
-}
-.fa-trash-o:before {
-  content: "\f014";
-}
-.fa-home:before {
-  content: "\f015";
-}
-.fa-file-o:before {
-  content: "\f016";
-}
-.fa-clock-o:before {
-  content: "\f017";
-}
-.fa-road:before {
-  content: "\f018";
-}
-.fa-download:before {
-  content: "\f019";
-}
-.fa-arrow-circle-o-down:before {
-  content: "\f01a";
-}
-.fa-arrow-circle-o-up:before {
-  content: "\f01b";
-}
-.fa-inbox:before {
-  content: "\f01c";
-}
-.fa-play-circle-o:before {
-  content: "\f01d";
-}
-.fa-rotate-right:before,
-.fa-repeat:before {
-  content: "\f01e";
-}
-.fa-refresh:before {
-  content: "\f021";
-}
-.fa-list-alt:before {
-  content: "\f022";
-}
-.fa-lock:before {
-  content: "\f023";
-}
-.fa-flag:before {
-  content: "\f024";
-}
-.fa-headphones:before {
-  content: "\f025";
-}
-.fa-volume-off:before {
-  content: "\f026";
-}
-.fa-volume-down:before {
-  content: "\f027";
-}
-.fa-volume-up:before {
-  content: "\f028";
-}
-.fa-qrcode:before {
-  content: "\f029";
-}
-.fa-barcode:before {
-  content: "\f02a";
-}
-.fa-tag:before {
-  content: "\f02b";
-}
-.fa-tags:before {
-  content: "\f02c";
-}
-.fa-book:before {
-  content: "\f02d";
-}
-.fa-bookmark:before {
-  content: "\f02e";
-}
-.fa-print:before {
-  content: "\f02f";
-}
-.fa-camera:before {
-  content: "\f030";
-}
-.fa-font:before {
-  content: "\f031";
-}
-.fa-bold:before {
-  content: "\f032";
-}
-.fa-italic:before {
-  content: "\f033";
-}
-.fa-text-height:before {
-  content: "\f034";
-}
-.fa-text-width:before {
-  content: "\f035";
-}
-.fa-align-left:before {
-  content: "\f036";
-}
-.fa-align-center:before {
-  content: "\f037";
-}
-.fa-align-right:before {
-  content: "\f038";
-}
-.fa-align-justify:before {
-  content: "\f039";
-}
-.fa-list:before {
-  content: "\f03a";
-}
-.fa-dedent:before,
-.fa-outdent:before {
-  content: "\f03b";
-}
-.fa-indent:before {
-  content: "\f03c";
-}
-.fa-video-camera:before {
-  content: "\f03d";
-}
-.fa-photo:before,
-.fa-image:before,
-.fa-picture-o:before {
-  content: "\f03e";
-}
-.fa-pencil:before {
-  content: "\f040";
-}
-.fa-map-marker:before {
-  content: "\f041";
-}
-.fa-adjust:before {
-  content: "\f042";
-}
-.fa-tint:before {
-  content: "\f043";
-}
-.fa-edit:before,
-.fa-pencil-square-o:before {
-  content: "\f044";
-}
-.fa-share-square-o:before {
-  content: "\f045";
-}
-.fa-check-square-o:before {
-  content: "\f046";
-}
-.fa-arrows:before {
-  content: "\f047";
-}
-.fa-step-backward:before {
-  content: "\f048";
-}
-.fa-fast-backward:before {
-  content: "\f049";
-}
-.fa-backward:before {
-  content: "\f04a";
-}
-.fa-play:before {
-  content: "\f04b";
-}
-.fa-pause:before {
-  content: "\f04c";
-}
-.fa-stop:before {
-  content: "\f04d";
-}
-.fa-forward:before {
-  content: "\f04e";
-}
-.fa-fast-forward:before {
-  content: "\f050";
-}
-.fa-step-forward:before {
-  content: "\f051";
-}
-.fa-eject:before {
-  content: "\f052";
-}
-.fa-chevron-left:before {
-  content: "\f053";
-}
-.fa-chevron-right:before {
-  content: "\f054";
-}
-.fa-plus-circle:before {
-  content: "\f055";
-}
-.fa-minus-circle:before {
-  content: "\f056";
-}
-.fa-times-circle:before {
-  content: "\f057";
-}
-.fa-check-circle:before {
-  content: "\f058";
-}
-.fa-question-circle:before {
-  content: "\f059";
-}
-.fa-info-circle:before {
-  content: "\f05a";
-}
-.fa-crosshairs:before {
-  content: "\f05b";
-}
-.fa-times-circle-o:before {
-  content: "\f05c";
-}
-.fa-check-circle-o:before {
-  content: "\f05d";
-}
-.fa-ban:before {
-  content: "\f05e";
-}
-.fa-arrow-left:before {
-  content: "\f060";
-}
-.fa-arrow-right:before {
-  content: "\f061";
-}
-.fa-arrow-up:before {
-  content: "\f062";
-}
-.fa-arrow-down:before {
-  content: "\f063";
-}
-.fa-mail-forward:before,
-.fa-share:before {
-  content: "\f064";
-}
-.fa-expand:before {
-  content: "\f065";
-}
-.fa-compress:before {
-  content: "\f066";
-}
-.fa-plus:before {
-  content: "\f067";
-}
-.fa-minus:before {
-  content: "\f068";
-}
-.fa-asterisk:before {
-  content: "\f069";
-}
-.fa-exclamation-circle:before {
-  content: "\f06a";
-}
-.fa-gift:before {
-  content: "\f06b";
-}
-.fa-leaf:before {
-  content: "\f06c";
-}
-.fa-fire:before {
-  content: "\f06d";
-}
-.fa-eye:before {
-  content: "\f06e";
-}
-.fa-eye-slash:before {
-  content: "\f070";
-}
-.fa-warning:before,
-.fa-exclamation-triangle:before {
-  content: "\f071";
-}
-.fa-plane:before {
-  content: "\f072";
-}
-.fa-calendar:before {
-  content: "\f073";
-}
-.fa-random:before {
-  content: "\f074";
-}
-.fa-comment:before {
-  content: "\f075";
-}
-.fa-magnet:before {
-  content: "\f076";
-}
-.fa-chevron-up:before {
-  content: "\f077";
-}
-.fa-chevron-down:before {
-  content: "\f078";
-}
-.fa-retweet:before {
-  content: "\f079";
-}
-.fa-shopping-cart:before {
-  content: "\f07a";
-}
-.fa-folder:before {
-  content: "\f07b";
-}
-.fa-folder-open:before {
-  content: "\f07c";
-}
-.fa-arrows-v:before {
-  content: "\f07d";
-}
-.fa-arrows-h:before {
-  content: "\f07e";
-}
-.fa-bar-chart-o:before,
-.fa-bar-chart:before {
-  content: "\f080";
-}
-.fa-twitter-square:before {
-  content: "\f081";
-}
-.fa-facebook-square:before {
-  content: "\f082";
-}
-.fa-camera-retro:before {
-  content: "\f083";
-}
-.fa-key:before {
-  content: "\f084";
-}
-.fa-gears:before,
-.fa-cogs:before {
-  content: "\f085";
-}
-.fa-comments:before {
-  content: "\f086";
-}
-.fa-thumbs-o-up:before {
-  content: "\f087";
-}
-.fa-thumbs-o-down:before {
-  content: "\f088";
-}
-.fa-star-half:before {
-  content: "\f089";
-}
-.fa-heart-o:before {
-  content: "\f08a";
-}
-.fa-sign-out:before {
-  content: "\f08b";
-}
-.fa-linkedin-square:before {
-  content: "\f08c";
-}
-.fa-thumb-tack:before {
-  content: "\f08d";
-}
-.fa-external-link:before {
-  content: "\f08e";
-}
-.fa-sign-in:before {
-  content: "\f090";
-}
-.fa-trophy:before {
-  content: "\f091";
-}
-.fa-github-square:before {
-  content: "\f092";
-}
-.fa-upload:before {
-  content: "\f093";
-}
-.fa-lemon-o:before {
-  content: "\f094";
-}
-.fa-phone:before {
-  content: "\f095";
-}
-.fa-square-o:before {
-  content: "\f096";
-}
-.fa-bookmark-o:before {
-  content: "\f097";
-}
-.fa-phone-square:before {
-  content: "\f098";
-}
-.fa-twitter:before {
-  content: "\f099";
-}
-.fa-facebook-f:before,
-.fa-facebook:before {
-  content: "\f09a";
-}
-.fa-github:before {
-  content: "\f09b";
-}
-.fa-unlock:before {
-  content: "\f09c";
-}
-.fa-credit-card:before {
-  content: "\f09d";
-}
-.fa-rss:before {
-  content: "\f09e";
-}
-.fa-hdd-o:before {
-  content: "\f0a0";
-}
-.fa-bullhorn:before {
-  content: "\f0a1";
-}
-.fa-bell:before {
-  content: "\f0f3";
-}
-.fa-certificate:before {
-  content: "\f0a3";
-}
-.fa-hand-o-right:before {
-  content: "\f0a4";
-}
-.fa-hand-o-left:before {
-  content: "\f0a5";
-}
-.fa-hand-o-up:before {
-  content: "\f0a6";
-}
-.fa-hand-o-down:before {
-  content: "\f0a7";
-}
-.fa-arrow-circle-left:before {
-  content: "\f0a8";
-}
-.fa-arrow-circle-right:before {
-  content: "\f0a9";
-}
-.fa-arrow-circle-up:before {
-  content: "\f0aa";
-}
-.fa-arrow-circle-down:before {
-  content: "\f0ab";
-}
-.fa-globe:before {
-  content: "\f0ac";
-}
-.fa-wrench:before {
-  content: "\f0ad";
-}
-.fa-tasks:before {
-  content: "\f0ae";
-}
-.fa-filter:before {
-  content: "\f0b0";
-}
-.fa-briefcase:before {
-  content: "\f0b1";
-}
-.fa-arrows-alt:before {
-  content: "\f0b2";
-}
-.fa-group:before,
-.fa-users:before {
-  content: "\f0c0";
-}
-.fa-chain:before,
-.fa-link:before {
-  content: "\f0c1";
-}
-.fa-cloud:before {
-  content: "\f0c2";
-}
-.fa-flask:before {
-  content: "\f0c3";
-}
-.fa-cut:before,
-.fa-scissors:before {
-  content: "\f0c4";
-}
-.fa-copy:before,
-.fa-files-o:before {
-  content: "\f0c5";
-}
-.fa-paperclip:before {
-  content: "\f0c6";
-}
-.fa-save:before,
-.fa-floppy-o:before {
-  content: "\f0c7";
-}
-.fa-square:before {
-  content: "\f0c8";
-}
-.fa-navicon:before,
-.fa-reorder:before,
-.fa-bars:before {
-  content: "\f0c9";
-}
-.fa-list-ul:before {
-  content: "\f0ca";
-}
-.fa-list-ol:before {
-  content: "\f0cb";
-}
-.fa-strikethrough:before {
-  content: "\f0cc";
-}
-.fa-underline:before {
-  content: "\f0cd";
-}
-.fa-table:before {
-  content: "\f0ce";
-}
-.fa-magic:before {
-  content: "\f0d0";
-}
-.fa-truck:before {
-  content: "\f0d1";
-}
-.fa-pinterest:before {
-  content: "\f0d2";
-}
-.fa-pinterest-square:before {
-  content: "\f0d3";
-}
-.fa-google-plus-square:before {
-  content: "\f0d4";
-}
-.fa-google-plus:before {
-  content: "\f0d5";
-}
-.fa-money:before {
-  content: "\f0d6";
-}
-.fa-caret-down:before {
-  content: "\f0d7";
-}
-.fa-caret-up:before {
-  content: "\f0d8";
-}
-.fa-caret-left:before {
-  content: "\f0d9";
-}
-.fa-caret-right:before {
-  content: "\f0da";
-}
-.fa-columns:before {
-  content: "\f0db";
-}
-.fa-unsorted:before,
-.fa-sort:before {
-  content: "\f0dc";
-}
-.fa-sort-down:before,
-.fa-sort-desc:before {
-  content: "\f0dd";
-}
-.fa-sort-up:before,
-.fa-sort-asc:before {
-  content: "\f0de";
-}
-.fa-envelope:before {
-  content: "\f0e0";
-}
-.fa-linkedin:before {
-  content: "\f0e1";
-}
-.fa-rotate-left:before,
-.fa-undo:before {
-  content: "\f0e2";
-}
-.fa-legal:before,
-.fa-gavel:before {
-  content: "\f0e3";
-}
-.fa-dashboard:before,
-.fa-tachometer:before {
-  content: "\f0e4";
-}
-.fa-comment-o:before {
-  content: "\f0e5";
-}
-.fa-comments-o:before {
-  content: "\f0e6";
-}
-.fa-flash:before,
-.fa-bolt:before {
-  content: "\f0e7";
-}
-.fa-sitemap:before {
-  content: "\f0e8";
-}
-.fa-umbrella:before {
-  content: "\f0e9";
-}
-.fa-paste:before,
-.fa-clipboard:before {
-  content: "\f0ea";
-}
-.fa-lightbulb-o:before {
-  content: "\f0eb";
-}
-.fa-exchange:before {
-  content: "\f0ec";
-}
-.fa-cloud-download:before {
-  content: "\f0ed";
-}
-.fa-cloud-upload:before {
-  content: "\f0ee";
-}
-.fa-user-md:before {
-  content: "\f0f0";
-}
-.fa-stethoscope:before {
-  content: "\f0f1";
-}
-.fa-suitcase:before {
-  content: "\f0f2";
-}
-.fa-bell-o:before {
-  content: "\f0a2";
-}
-.fa-coffee:before {
-  content: "\f0f4";
-}
-.fa-cutlery:before {
-  content: "\f0f5";
-}
-.fa-file-text-o:before {
-  content: "\f0f6";
-}
-.fa-building-o:before {
-  content: "\f0f7";
-}
-.fa-hospital-o:before {
-  content: "\f0f8";
-}
-.fa-ambulance:before {
-  content: "\f0f9";
-}
-.fa-medkit:before {
-  content: "\f0fa";
-}
-.fa-fighter-jet:before {
-  content: "\f0fb";
-}
-.fa-beer:before {
-  content: "\f0fc";
-}
-.fa-h-square:before {
-  content: "\f0fd";
-}
-.fa-plus-square:before {
-  content: "\f0fe";
-}
-.fa-angle-double-left:before {
-  content: "\f100";
-}
-.fa-angle-double-right:before {
-  content: "\f101";
-}
-.fa-angle-double-up:before {
-  content: "\f102";
-}
-.fa-angle-double-down:before {
-  content: "\f103";
-}
-.fa-angle-left:before {
-  content: "\f104";
-}
-.fa-angle-right:before {
-  content: "\f105";
-}
-.fa-angle-up:before {
-  content: "\f106";
-}
-.fa-angle-down:before {
-  content: "\f107";
-}
-.fa-desktop:before {
-  content: "\f108";
-}
-.fa-laptop:before {
-  content: "\f109";
-}
-.fa-tablet:before {
-  content: "\f10a";
-}
-.fa-mobile-phone:before,
-.fa-mobile:before {
-  content: "\f10b";
-}
-.fa-circle-o:before {
-  content: "\f10c";
-}
-.fa-quote-left:before {
-  content: "\f10d";
-}
-.fa-quote-right:before {
-  content: "\f10e";
-}
-.fa-spinner:before {
-  content: "\f110";
-}
-.fa-circle:before {
-  content: "\f111";
-}
-.fa-mail-reply:before,
-.fa-reply:before {
-  content: "\f112";
-}
-.fa-github-alt:before {
-  content: "\f113";
-}
-.fa-folder-o:before {
-  content: "\f114";
-}
-.fa-folder-open-o:before {
-  content: "\f115";
-}
-.fa-smile-o:before {
-  content: "\f118";
-}
-.fa-frown-o:before {
-  content: "\f119";
-}
-.fa-meh-o:before {
-  content: "\f11a";
-}
-.fa-gamepad:before {
-  content: "\f11b";
-}
-.fa-keyboard-o:before {
-  content: "\f11c";
-}
-.fa-flag-o:before {
-  content: "\f11d";
-}
-.fa-flag-checkered:before {
-  content: "\f11e";
-}
-.fa-terminal:before {
-  content: "\f120";
-}
-.fa-code:before {
-  content: "\f121";
-}
-.fa-mail-reply-all:before,
-.fa-reply-all:before {
-  content: "\f122";
-}
-.fa-star-half-empty:before,
-.fa-star-half-full:before,
-.fa-star-half-o:before {
-  content: "\f123";
-}
-.fa-location-arrow:before {
-  content: "\f124";
-}
-.fa-crop:before {
-  content: "\f125";
-}
-.fa-code-fork:before {
-  content: "\f126";
-}
-.fa-unlink:before,
-.fa-chain-broken:before {
-  content: "\f127";
-}
-.fa-question:before {
-  content: "\f128";
-}
-.fa-info:before {
-  content: "\f129";
-}
-.fa-exclamation:before {
-  content: "\f12a";
-}
-.fa-superscript:before {
-  content: "\f12b";
-}
-.fa-subscript:before {
-  content: "\f12c";
-}
-.fa-eraser:before {
-  content: "\f12d";
-}
-.fa-puzzle-piece:before {
-  content: "\f12e";
-}
-.fa-microphone:before {
-  content: "\f130";
-}
-.fa-microphone-slash:before {
-  content: "\f131";
-}
-.fa-shield:before {
-  content: "\f132";
-}
-.fa-calendar-o:before {
-  content: "\f133";
-}
-.fa-fire-extinguisher:before {
-  content: "\f134";
-}
-.fa-rocket:before {
-  content: "\f135";
-}
-.fa-maxcdn:before {
-  content: "\f136";
-}
-.fa-chevron-circle-left:before {
-  content: "\f137";
-}
-.fa-chevron-circle-right:before {
-  content: "\f138";
-}
-.fa-chevron-circle-up:before {
-  content: "\f139";
-}
-.fa-chevron-circle-down:before {
-  content: "\f13a";
-}
-.fa-html5:before {
-  content: "\f13b";
-}
-.fa-css3:before {
-  content: "\f13c";
-}
-.fa-anchor:before {
-  content: "\f13d";
-}
-.fa-unlock-alt:before {
-  content: "\f13e";
-}
-.fa-bullseye:before {
-  content: "\f140";
-}
-.fa-ellipsis-h:before {
-  content: "\f141";
-}
-.fa-ellipsis-v:before {
-  content: "\f142";
-}
-.fa-rss-square:before {
-  content: "\f143";
-}
-.fa-play-circle:before {
-  content: "\f144";
-}
-.fa-ticket:before {
-  content: "\f145";
-}
-.fa-minus-square:before {
-  content: "\f146";
-}
-.fa-minus-square-o:before {
-  content: "\f147";
-}
-.fa-level-up:before {
-  content: "\f148";
-}
-.fa-level-down:before {
-  content: "\f149";
-}
-.fa-check-square:before {
-  content: "\f14a";
-}
-.fa-pencil-square:before {
-  content: "\f14b";
-}
-.fa-external-link-square:before {
-  content: "\f14c";
-}
-.fa-share-square:before {
-  content: "\f14d";
-}
-.fa-compass:before {
-  content: "\f14e";
-}
-.fa-toggle-down:before,
-.fa-caret-square-o-down:before {
-  content: "\f150";
-}
-.fa-toggle-up:before,
-.fa-caret-square-o-up:before {
-  content: "\f151";
-}
-.fa-toggle-right:before,
-.fa-caret-square-o-right:before {
-  content: "\f152";
-}
-.fa-euro:before,
-.fa-eur:before {
-  content: "\f153";
-}
-.fa-gbp:before {
-  content: "\f154";
-}
-.fa-dollar:before,
-.fa-usd:before {
-  content: "\f155";
-}
-.fa-rupee:before,
-.fa-inr:before {
-  content: "\f156";
-}
-.fa-cny:before,
-.fa-rmb:before,
-.fa-yen:before,
-.fa-jpy:before {
-  content: "\f157";
-}
-.fa-ruble:before,
-.fa-rouble:before,
-.fa-rub:before {
-  content: "\f158";
-}
-.fa-won:before,
-.fa-krw:before {
-  content: "\f159";
-}
-.fa-bitcoin:before,
-.fa-btc:before {
-  content: "\f15a";
-}
-.fa-file:before {
-  content: "\f15b";
-}
-.fa-file-text:before {
-  content: "\f15c";
-}
-.fa-sort-alpha-asc:before {
-  content: "\f15d";
-}
-.fa-sort-alpha-desc:before {
-  content: "\f15e";
-}
-.fa-sort-amount-asc:before {
-  content: "\f160";
-}
-.fa-sort-amount-desc:before {
-  content: "\f161";
-}
-.fa-sort-numeric-asc:before {
-  content: "\f162";
-}
-.fa-sort-numeric-desc:before {
-  content: "\f163";
-}
-.fa-thumbs-up:before {
-  content: "\f164";
-}
-.fa-thumbs-down:before {
-  content: "\f165";
-}
-.fa-youtube-square:before {
-  content: "\f166";
-}
-.fa-youtube:before {
-  content: "\f167";
-}
-.fa-xing:before {
-  content: "\f168";
-}
-.fa-xing-square:before {
-  content: "\f169";
-}
-.fa-youtube-play:before {
-  content: "\f16a";
-}
-.fa-dropbox:before {
-  content: "\f16b";
-}
-.fa-stack-overflow:before {
-  content: "\f16c";
-}
-.fa-instagram:before {
-  content: "\f16d";
-}
-.fa-flickr:before {
-  content: "\f16e";
-}
-.fa-adn:before {
-  content: "\f170";
-}
-.fa-bitbucket:before {
-  content: "\f171";
-}
-.fa-bitbucket-square:before {
-  content: "\f172";
-}
-.fa-tumblr:before {
-  content: "\f173";
-}
-.fa-tumblr-square:before {
-  content: "\f174";
-}
-.fa-long-arrow-down:before {
-  content: "\f175";
-}
-.fa-long-arrow-up:before {
-  content: "\f176";
-}
-.fa-long-arrow-left:before {
-  content: "\f177";
-}
-.fa-long-arrow-right:before {
-  content: "\f178";
-}
-.fa-apple:before {
-  content: "\f179";
-}
-.fa-windows:before {
-  content: "\f17a";
-}
-.fa-android:before {
-  content: "\f17b";
-}
-.fa-linux:before {
-  content: "\f17c";
-}
-.fa-dribbble:before {
-  content: "\f17d";
-}
-.fa-skype:before {
-  content: "\f17e";
-}
-.fa-foursquare:before {
-  content: "\f180";
-}
-.fa-trello:before {
-  content: "\f181";
-}
-.fa-female:before {
-  content: "\f182";
-}
-.fa-male:before {
-  content: "\f183";
-}
-.fa-gittip:before,
-.fa-gratipay:before {
-  content: "\f184";
-}
-.fa-sun-o:before {
-  content: "\f185";
-}
-.fa-moon-o:before {
-  content: "\f186";
-}
-.fa-archive:before {
-  content: "\f187";
-}
-.fa-bug:before {
-  content: "\f188";
-}
-.fa-vk:before {
-  content: "\f189";
-}
-.fa-weibo:before {
-  content: "\f18a";
-}
-.fa-renren:before {
-  content: "\f18b";
-}
-.fa-pagelines:before {
-  content: "\f18c";
-}
-.fa-stack-exchange:before {
-  content: "\f18d";
-}
-.fa-arrow-circle-o-right:before {
-  content: "\f18e";
-}
-.fa-arrow-circle-o-left:before {
-  content: "\f190";
-}
-.fa-toggle-left:before,
-.fa-caret-square-o-left:before {
-  content: "\f191";
-}
-.fa-dot-circle-o:before {
-  content: "\f192";
-}
-.fa-wheelchair:before {
-  content: "\f193";
-}
-.fa-vimeo-square:before {
-  content: "\f194";
-}
-.fa-turkish-lira:before,
-.fa-try:before {
-  content: "\f195";
-}
-.fa-plus-square-o:before {
-  content: "\f196";
-}
-.fa-space-shuttle:before {
-  content: "\f197";
-}
-.fa-slack:before {
-  content: "\f198";
-}
-.fa-envelope-square:before {
-  content: "\f199";
-}
-.fa-wordpress:before {
-  content: "\f19a";
-}
-.fa-openid:before {
-  content: "\f19b";
-}
-.fa-institution:before,
-.fa-bank:before,
-.fa-university:before {
-  content: "\f19c";
-}
-.fa-mortar-board:before,
-.fa-graduation-cap:before {
-  content: "\f19d";
-}
-.fa-yahoo:before {
-  content: "\f19e";
-}
-.fa-google:before {
-  content: "\f1a0";
-}
-.fa-reddit:before {
-  content: "\f1a1";
-}
-.fa-reddit-square:before {
-  content: "\f1a2";
-}
-.fa-stumbleupon-circle:before {
-  content: "\f1a3";
-}
-.fa-stumbleupon:before {
-  content: "\f1a4";
-}
-.fa-delicious:before {
-  content: "\f1a5";
-}
-.fa-digg:before {
-  content: "\f1a6";
-}
-.fa-pied-piper:before {
-  content: "\f1a7";
-}
-.fa-pied-piper-alt:before {
-  content: "\f1a8";
-}
-.fa-drupal:before {
-  content: "\f1a9";
-}
-.fa-joomla:before {
-  content: "\f1aa";
-}
-.fa-language:before {
-  content: "\f1ab";
-}
-.fa-fax:before {
-  content: "\f1ac";
-}
-.fa-building:before {
-  content: "\f1ad";
-}
-.fa-child:before {
-  content: "\f1ae";
-}
-.fa-paw:before {
-  content: "\f1b0";
-}
-.fa-spoon:before {
-  content: "\f1b1";
-}
-.fa-cube:before {
-  content: "\f1b2";
-}
-.fa-cubes:before {
-  content: "\f1b3";
-}
-.fa-behance:before {
-  content: "\f1b4";
-}
-.fa-behance-square:before {
-  content: "\f1b5";
-}
-.fa-steam:before {
-  content: "\f1b6";
-}
-.fa-steam-square:before {
-  content: "\f1b7";
-}
-.fa-recycle:before {
-  content: "\f1b8";
-}
-.fa-automobile:before,
-.fa-car:before {
-  content: "\f1b9";
-}
-.fa-cab:before,
-.fa-taxi:before {
-  content: "\f1ba";
-}
-.fa-tree:before {
-  content: "\f1bb";
-}
-.fa-spotify:before {
-  content: "\f1bc";
-}
-.fa-deviantart:before {
-  content: "\f1bd";
-}
-.fa-soundcloud:before {
-  content: "\f1be";
-}
-.fa-database:before {
-  content: "\f1c0";
-}
-.fa-file-pdf-o:before {
-  content: "\f1c1";
-}
-.fa-file-word-o:before {
-  content: "\f1c2";
-}
-.fa-file-excel-o:before {
-  content: "\f1c3";
-}
-.fa-file-powerpoint-o:before {
-  content: "\f1c4";
-}
-.fa-file-photo-o:before,
-.fa-file-picture-o:before,
-.fa-file-image-o:before {
-  content: "\f1c5";
-}
-.fa-file-zip-o:before,
-.fa-file-archive-o:before {
-  content: "\f1c6";
-}
-.fa-file-sound-o:before,
-.fa-file-audio-o:before {
-  content: "\f1c7";
-}
-.fa-file-movie-o:before,
-.fa-file-video-o:before {
-  content: "\f1c8";
-}
-.fa-file-code-o:before {
-  content: "\f1c9";
-}
-.fa-vine:before {
-  content: "\f1ca";
-}
-.fa-codepen:before {
-  content: "\f1cb";
-}
-.fa-jsfiddle:before {
-  content: "\f1cc";
-}
-.fa-life-bouy:before,
-.fa-life-buoy:before,
-.fa-life-saver:before,
-.fa-support:before,
-.fa-life-ring:before {
-  content: "\f1cd";
-}
-.fa-circle-o-notch:before {
-  content: "\f1ce";
-}
-.fa-ra:before,
-.fa-rebel:before {
-  content: "\f1d0";
-}
-.fa-ge:before,
-.fa-empire:before {
-  content: "\f1d1";
-}
-.fa-git-square:before {
-  content: "\f1d2";
-}
-.fa-git:before {
-  content: "\f1d3";
-}
-.fa-hacker-news:before {
-  content: "\f1d4";
-}
-.fa-tencent-weibo:before {
-  content: "\f1d5";
-}
-.fa-qq:before {
-  content: "\f1d6";
-}
-.fa-wechat:before,
-.fa-weixin:before {
-  content: "\f1d7";
-}
-.fa-send:before,
-.fa-paper-plane:before {
-  content: "\f1d8";
-}
-.fa-send-o:before,
-.fa-paper-plane-o:before {
-  content: "\f1d9";
-}
-.fa-history:before {
-  content: "\f1da";
-}
-.fa-genderless:before,
-.fa-circle-thin:before {
-  content: "\f1db";
-}
-.fa-header:before {
-  content: "\f1dc";
-}
-.fa-paragraph:before {
-  content: "\f1dd";
-}
-.fa-sliders:before {
-  content: "\f1de";
-}
-.fa-share-alt:before {
-  content: "\f1e0";
-}
-.fa-share-alt-square:before {
-  content: "\f1e1";
-}
-.fa-bomb:before {
-  content: "\f1e2";
-}
-.fa-soccer-ball-o:before,
-.fa-futbol-o:before {
-  content: "\f1e3";
-}
-.fa-tty:before {
-  content: "\f1e4";
-}
-.fa-binoculars:before {
-  content: "\f1e5";
-}
-.fa-plug:before {
-  content: "\f1e6";
-}
-.fa-slideshare:before {
-  content: "\f1e7";
-}
-.fa-twitch:before {
-  content: "\f1e8";
-}
-.fa-yelp:before {
-  content: "\f1e9";
-}
-.fa-newspaper-o:before {
-  content: "\f1ea";
-}
-.fa-wifi:before {
-  content: "\f1eb";
-}
-.fa-calculator:before {
-  content: "\f1ec";
-}
-.fa-paypal:before {
-  content: "\f1ed";
-}
-.fa-google-wallet:before {
-  content: "\f1ee";
-}
-.fa-cc-visa:before {
-  content: "\f1f0";
-}
-.fa-cc-mastercard:before {
-  content: "\f1f1";
-}
-.fa-cc-discover:before {
-  content: "\f1f2";
-}
-.fa-cc-amex:before {
-  content: "\f1f3";
-}
-.fa-cc-paypal:before {
-  content: "\f1f4";
-}
-.fa-cc-stripe:before {
-  content: "\f1f5";
-}
-.fa-bell-slash:before {
-  content: "\f1f6";
-}
-.fa-bell-slash-o:before {
-  content: "\f1f7";
-}
-.fa-trash:before {
-  content: "\f1f8";
-}
-.fa-copyright:before {
-  content: "\f1f9";
-}
-.fa-at:before {
-  content: "\f1fa";
-}
-.fa-eyedropper:before {
-  content: "\f1fb";
-}
-.fa-paint-brush:before {
-  content: "\f1fc";
-}
-.fa-birthday-cake:before {
-  content: "\f1fd";
-}
-.fa-area-chart:before {
-  content: "\f1fe";
-}
-.fa-pie-chart:before {
-  content: "\f200";
-}
-.fa-line-chart:before {
-  content: "\f201";
-}
-.fa-lastfm:before {
-  content: "\f202";
-}
-.fa-lastfm-square:before {
-  content: "\f203";
-}
-.fa-toggle-off:before {
-  content: "\f204";
-}
-.fa-toggle-on:before {
-  content: "\f205";
-}
-.fa-bicycle:before {
-  content: "\f206";
-}
-.fa-bus:before {
-  content: "\f207";
-}
-.fa-ioxhost:before {
-  content: "\f208";
-}
-.fa-angellist:before {
-  content: "\f209";
-}
-.fa-cc:before {
-  content: "\f20a";
-}
-.fa-shekel:before,
-.fa-sheqel:before,
-.fa-ils:before {
-  content: "\f20b";
-}
-.fa-meanpath:before {
-  content: "\f20c";
-}
-.fa-buysellads:before {
-  content: "\f20d";
-}
-.fa-connectdevelop:before {
-  content: "\f20e";
-}
-.fa-dashcube:before {
-  content: "\f210";
-}
-.fa-forumbee:before {
-  content: "\f211";
-}
-.fa-leanpub:before {
-  content: "\f212";
-}
-.fa-sellsy:before {
-  content: "\f213";
-}
-.fa-shirtsinbulk:before {
-  content: "\f214";
-}
-.fa-simplybuilt:before {
-  content: "\f215";
-}
-.fa-skyatlas:before {
-  content: "\f216";
-}
-.fa-cart-plus:before {
-  content: "\f217";
-}
-.fa-cart-arrow-down:before {
-  content: "\f218";
-}
-.fa-diamond:before {
-  content: "\f219";
-}
-.fa-ship:before {
-  content: "\f21a";
-}
-.fa-user-secret:before {
-  content: "\f21b";
-}
-.fa-motorcycle:before {
-  content: "\f21c";
-}
-.fa-street-view:before {
-  content: "\f21d";
-}
-.fa-heartbeat:before {
-  content: "\f21e";
-}
-.fa-venus:before {
-  content: "\f221";
-}
-.fa-mars:before {
-  content: "\f222";
-}
-.fa-mercury:before {
-  content: "\f223";
-}
-.fa-transgender:before {
-  content: "\f224";
-}
-.fa-transgender-alt:before {
-  content: "\f225";
-}
-.fa-venus-double:before {
-  content: "\f226";
-}
-.fa-mars-double:before {
-  content: "\f227";
-}
-.fa-venus-mars:before {
-  content: "\f228";
-}
-.fa-mars-stroke:before {
-  content: "\f229";
-}
-.fa-mars-stroke-v:before {
-  content: "\f22a";
-}
-.fa-mars-stroke-h:before {
-  content: "\f22b";
-}
-.fa-neuter:before {
-  content: "\f22c";
-}
-.fa-facebook-official:before {
-  content: "\f230";
-}
-.fa-pinterest-p:before {
-  content: "\f231";
-}
-.fa-whatsapp:before {
-  content: "\f232";
-}
-.fa-server:before {
-  content: "\f233";
-}
-.fa-user-plus:before {
-  content: "\f234";
-}
-.fa-user-times:before {
-  content: "\f235";
-}
-.fa-hotel:before,
-.fa-bed:before {
-  content: "\f236";
-}
-.fa-viacoin:before {
-  content: "\f237";
-}
-.fa-train:before {
-  content: "\f238";
-}
-.fa-subway:before {
-  content: "\f239";
-}
-.fa-medium:before {
-  content: "\f23a";
-}
-
-/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */
-html {
-  font-family: sans-serif;
-  -ms-text-size-adjust: 100%;
-  -webkit-text-size-adjust: 100%;
-}
-body {
-  margin: 0;
-}
-article,
-aside,
-details,
-figcaption,
-figure,
-footer,
-header,
-hgroup,
-main,
-menu,
-nav,
-section,
-summary {
-  display: block;
-}
-audio,
-canvas,
-progress,
-video {
-  display: inline-block;
-  vertical-align: baseline;
-}
-audio:not([controls]) {
-  display: none;
-  height: 0;
-}
-[hidden],
-template {
-  display: none;
-}
-a {
-  background-color: transparent;
-}
-a:active,
-a:hover {
-  outline: 0;
-}
-abbr[title] {
-  border-bottom: 1px dotted;
-}
-b,
-strong {
-  font-weight: bold;
-}
-dfn {
-  font-style: italic;
-}
-h1 {
-  font-size: 2em;
-  margin: 0.67em 0;
-}
-mark {
-  background: #ff0;
-  color: #000;
-}
-small {
-  font-size: 80%;
-}
-sub,
-sup {
-  font-size: 75%;
-  line-height: 0;
-  position: relative;
-  vertical-align: baseline;
-}
-sup {
-  top: -0.5em;
-}
-sub {
-  bottom: -0.25em;
-}
-img {
-  border: 0;
-}
-svg:not(:root) {
-  overflow: hidden;
-}
-figure {
-  margin: 1em 40px;
-}
-hr {
-  box-sizing: content-box;
-  height: 0;
-}
-pre {
-  overflow: auto;
-}
-code,
-kbd,
-pre,
-samp {
-  font-family: monospace, monospace;
-  font-size: 1em;
-}
-button,
-input,
-optgroup,
-select,
-textarea {
-  color: inherit;
-  font: inherit;
-  margin: 0;
-}
-button {
-  overflow: visible;
-}
-button,
-select {
-  text-transform: none;
-}
-button,
-html input[type="button"],
-input[type="reset"],
-input[type="submit"] {
-  -webkit-appearance: button;
-  cursor: pointer;
-}
-button[disabled],
-html input[disabled] {
-  cursor: default;
-}
-button::-moz-focus-inner,
-input::-moz-focus-inner {
-  border: 0;
-  padding: 0;
-}
-input {
-  line-height: normal;
-}
-input[type="checkbox"],
-input[type="radio"] {
-  box-sizing: border-box;
-  padding: 0;
-}
-input[type="number"]::-webkit-inner-spin-button,
-input[type="number"]::-webkit-outer-spin-button {
-  height: auto;
-}
-input[type="search"] {
-  -webkit-appearance: textfield;
-  box-sizing: content-box;
-}
-input[type="search"]::-webkit-search-cancel-button,
-input[type="search"]::-webkit-search-decoration {
-  -webkit-appearance: none;
-}
-fieldset {
-  border: 1px solid #c0c0c0;
-  margin: 0 2px;
-  padding: 0.35em 0.625em 0.75em;
-}
-legend {
-  border: 0;
-  padding: 0;
-}
-textarea {
-  overflow: auto;
-}
-optgroup {
-  font-weight: bold;
-}
-table {
-  border-collapse: collapse;
-  border-spacing: 0;
-}
-td,
-th {
-  padding: 0;
-}
-/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */
-@media print {
-  *,
-  *:before,
-  *:after {
-    background: transparent !important;
-    color: #000 !important;
-    box-shadow: none !important;
-    text-shadow: none !important;
-  }
-  a,
-  a:visited {
-    text-decoration: underline;
-  }
-  a[href]:after {
-    content: " (" attr(href) ")";
-  }
-  abbr[title]:after {
-    content: " (" attr(title) ")";
-  }
-  a[href^="#"]:after,
-  a[href^="javascript:"]:after {
-    content: "";
-  }
-  pre,
-  blockquote {
-    border: 1px solid #999;
-    page-break-inside: avoid;
-  }
-  thead {
-    display: table-header-group;
-  }
-  tr,
-  img {
-    page-break-inside: avoid;
-  }
-  img {
-    max-width: 100% !important;
-  }
-  p,
-  h2,
-  h3 {
-    orphans: 3;
-    widows: 3;
-  }
-  h2,
-  h3 {
-    page-break-after: avoid;
-  }
-  .navbar {
-    display: none;
-  }
-  .btn > .caret,
-  .dropup > .btn > .caret {
-    border-top-color: #000 !important;
-  }
-  .label {
-    border: 1px solid #000;
-  }
-  .table {
-    border-collapse: collapse !important;
-  }
-  .table td,
-  .table th {
-    background-color: #fff !important;
-  }
-  .table-bordered th,
-  .table-bordered td {
-    border: 1px solid #ddd !important;
-  }
-}
-@font-face {
-  font-family: 'Glyphicons Halflings';
-  src: url('../fonts/glyphicons-halflings-regular.eot');
-  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
-}
-.glyphicon {
-  position: relative;
-  top: 1px;
-  display: inline-block;
-  font-family: 'Glyphicons Halflings';
-  font-style: normal;
-  font-weight: normal;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-  -moz-osx-font-smoothing: grayscale;
-}
-.glyphicon-asterisk:before {
-  content: "\2a";
-}
-.glyphicon-plus:before {
-  content: "\2b";
-}
-.glyphicon-euro:before,
-.glyphicon-eur:before {
-  content: "\20ac";
-}
-.glyphicon-minus:before {
-  content: "\2212";
-}
-.glyphicon-cloud:before {
-  content: "\2601";
-}
-.glyphicon-envelope:before {
-  content: "\2709";
-}
-.glyphicon-pencil:before {
-  content: "\270f";
-}
-.glyphicon-glass:before {
-  content: "\e001";
-}
-.glyphicon-music:before {
-  content: "\e002";
-}
-.glyphicon-search:before {
-  content: "\e003";
-}
-.glyphicon-heart:before {
-  content: "\e005";
-}
-.glyphicon-star:before {
-  content: "\e006";
-}
-.glyphicon-star-empty:before {
-  content: "\e007";
-}
-.glyphicon-user:before {
-  content: "\e008";
-}
-.glyphicon-film:before {
-  content: "\e009";
-}
-.glyphicon-th-large:before {
-  content: "\e010";
-}
-.glyphicon-th:before {
-  content: "\e011";
-}
-.glyphicon-th-list:before {
-  content: "\e012";
-}
-.glyphicon-ok:before {
-  content: "\e013";
-}
-.glyphicon-remove:before {
-  content: "\e014";
-}
-.glyphicon-zoom-in:before {
-  content: "\e015";
-}
-.glyphicon-zoom-out:before {
-  content: "\e016";
-}
-.glyphicon-off:before {
-  content: "\e017";
-}
-.glyphicon-signal:before {
-  content: "\e018";
-}
-.glyphicon-cog:before {
-  content: "\e019";
-}
-.glyphicon-trash:before {
-  content: "\e020";
-}
-.glyphicon-home:before {
-  content: "\e021";
-}
-.glyphicon-file:before {
-  content: "\e022";
-}
-.glyphicon-time:before {
-  content: "\e023";
-}
-.glyphicon-road:before {
-  content: "\e024";
-}
-.glyphicon-download-alt:before {
-  content: "\e025";
-}
-.glyphicon-download:before {
-  content: "\e026";
-}
-.glyphicon-upload:before {
-  content: "\e027";
-}
-.glyphicon-inbox:before {
-  content: "\e028";
-}
-.glyphicon-play-circle:before {
-  content: "\e029";
-}
-.glyphicon-repeat:before {
-  content: "\e030";
-}
-.glyphicon-refresh:before {
-  content: "\e031";
-}
-.glyphicon-list-alt:before {
-  content: "\e032";
-}
-.glyphicon-lock:before {
-  content: "\e033";
-}
-.glyphicon-flag:before {
-  content: "\e034";
-}
-.glyphicon-headphones:before {
-  content: "\e035";
-}
-.glyphicon-volume-off:before {
-  content: "\e036";
-}
-.glyphicon-volume-down:before {
-  content: "\e037";
-}
-.glyphicon-volume-up:before {
-  content: "\e038";
-}
-.glyphicon-qrcode:before {
-  content: "\e039";
-}
-.glyphicon-barcode:before {
-  content: "\e040";
-}
-.glyphicon-tag:before {
-  content: "\e041";
-}
-.glyphicon-tags:before {
-  content: "\e042";
-}
-.glyphicon-book:before {
-  content: "\e043";
-}
-.glyphicon-bookmark:before {
-  content: "\e044";
-}
-.glyphicon-print:before {
-  content: "\e045";
-}
-.glyphicon-camera:before {
-  content: "\e046";
-}
-.glyphicon-font:before {
-  content: "\e047";
-}
-.glyphicon-bold:before {
-  content: "\e048";
-}
-.glyphicon-italic:before {
-  content: "\e049";
-}
-.glyphicon-text-height:before {
-  content: "\e050";
-}
-.glyphicon-text-width:before {
-  content: "\e051";
-}
-.glyphicon-align-left:before {
-  content: "\e052";
-}
-.glyphicon-align-center:before {
-  content: "\e053";
-}
-.glyphicon-align-right:before {
-  content: "\e054";
-}
-.glyphicon-align-justify:before {
-  content: "\e055";
-}
-.glyphicon-list:before {
-  content: "\e056";
-}
-.glyphicon-indent-left:before {
-  content: "\e057";
-}
-.glyphicon-indent-right:before {
-  content: "\e058";
-}
-.glyphicon-facetime-video:before {
-  content: "\e059";
-}
-.glyphicon-picture:before {
-  content: "\e060";
-}
-.glyphicon-map-marker:before {
-  content: "\e062";
-}
-.glyphicon-adjust:before {
-  content: "\e063";
-}
-.glyphicon-tint:before {
-  content: "\e064";
-}
-.glyphicon-edit:before {
-  content: "\e065";
-}
-.glyphicon-share:before {
-  content: "\e066";
-}
-.glyphicon-check:before {
-  content: "\e067";
-}
-.glyphicon-move:before {
-  content: "\e068";
-}
-.glyphicon-step-backward:before {
-  content: "\e069";
-}
-.glyphicon-fast-backward:before {
-  content: "\e070";
-}
-.glyphicon-backward:before {
-  content: "\e071";
-}
-.glyphicon-play:before {
-  content: "\e072";
-}
-.glyphicon-pause:before {
-  content: "\e073";
-}
-.glyphicon-stop:before {
-  content: "\e074";
-}
-.glyphicon-forward:before {
-  content: "\e075";
-}
-.glyphicon-fast-forward:before {
-  content: "\e076";
-}
-.glyphicon-step-forward:before {
-  content: "\e077";
-}
-.glyphicon-eject:before {
-  content: "\e078";
-}
-.glyphicon-chevron-left:before {
-  content: "\e079";
-}
-.glyphicon-chevron-right:before {
-  content: "\e080";
-}
-.glyphicon-plus-sign:before {
-  content: "\e081";
-}
-.glyphicon-minus-sign:before {
-  content: "\e082";
-}
-.glyphicon-remove-sign:before {
-  content: "\e083";
-}
-.glyphicon-ok-sign:before {
-  content: "\e084";
-}
-.glyphicon-question-sign:before {
-  content: "\e085";
-}
-.glyphicon-info-sign:before {
-  content: "\e086";
-}
-.glyphicon-screenshot:before {
-  content: "\e087";
-}
-.glyphicon-remove-circle:before {
-  content: "\e088";
-}
-.glyphicon-ok-circle:before {
-  content: "\e089";
-}
-.glyphicon-ban-circle:before {
-  content: "\e090";
-}
-.glyphicon-arrow-left:before {
-  content: "\e091";
-}
-.glyphicon-arrow-right:before {
-  content: "\e092";
-}
-.glyphicon-arrow-up:before {
-  content: "\e093";
-}
-.glyphicon-arrow-down:before {
-  content: "\e094";
-}
-.glyphicon-share-alt:before {
-  content: "\e095";
-}
-.glyphicon-resize-full:before {
-  content: "\e096";
-}
-.glyphicon-resize-small:before {
-  content: "\e097";
-}
-.glyphicon-exclamation-sign:before {
-  content: "\e101";
-}
-.glyphicon-gift:before {
-  content: "\e102";
-}
-.glyphicon-leaf:before {
-  content: "\e103";
-}
-.glyphicon-fire:before {
-  content: "\e104";
-}
-.glyphicon-eye-open:before {
-  content: "\e105";
-}
-.glyphicon-eye-close:before {
-  content: "\e106";
-}
-.glyphicon-warning-sign:before {
-  content: "\e107";
-}
-.glyphicon-plane:before {
-  content: "\e108";
-}
-.glyphicon-calendar:before {
-  content: "\e109";
-}
-.glyphicon-random:before {
-  content: "\e110";
-}
-.glyphicon-comment:before {
-  content: "\e111";
-}
-.glyphicon-magnet:before {
-  content: "\e112";
-}
-.glyphicon-chevron-up:before {
-  content: "\e113";
-}
-.glyphicon-chevron-down:before {
-  content: "\e114";
-}
-.glyphicon-retweet:before {
-  content: "\e115";
-}
-.glyphicon-shopping-cart:before {
-  content: "\e116";
-}
-.glyphicon-folder-close:before {
-  content: "\e117";
-}
-.glyphicon-folder-open:before {
-  content: "\e118";
-}
-.glyphicon-resize-vertical:before {
-  content: "\e119";
-}
-.glyphicon-resize-horizontal:before {
-  content: "\e120";
-}
-.glyphicon-hdd:before {
-  content: "\e121";
-}
-.glyphicon-bullhorn:before {
-  content: "\e122";
-}
-.glyphicon-bell:before {
-  content: "\e123";
-}
-.glyphicon-certificate:before {
-  content: "\e124";
-}
-.glyphicon-thumbs-up:before {
-  content: "\e125";
-}
-.glyphicon-thumbs-down:before {
-  content: "\e126";
-}
-.glyphicon-hand-right:before {
-  content: "\e127";
-}
-.glyphicon-hand-left:before {
-  content: "\e128";
-}
-.glyphicon-hand-up:before {
-  content: "\e129";
-}
-.glyphicon-hand-down:before {
-  content: "\e130";
-}
-.glyphicon-circle-arrow-right:before {
-  content: "\e131";
-}
-.glyphicon-circle-arrow-left:before {
-  content: "\e132";
-}
-.glyphicon-circle-arrow-up:before {
-  content: "\e133";
-}
-.glyphicon-circle-arrow-down:before {
-  content: "\e134";
-}
-.glyphicon-globe:before {
-  content: "\e135";
-}
-.glyphicon-wrench:before {
-  content: "\e136";
-}
-.glyphicon-tasks:before {
-  content: "\e137";
-}
-.glyphicon-filter:before {
-  content: "\e138";
-}
-.glyphicon-briefcase:before {
-  content: "\e139";
-}
-.glyphicon-fullscreen:before {
-  content: "\e140";
-}
-.glyphicon-dashboard:before {
-  content: "\e141";
-}
-.glyphicon-paperclip:before {
-  content: "\e142";
-}
-.glyphicon-heart-empty:before {
-  content: "\e143";
-}
-.glyphicon-link:before {
-  content: "\e144";
-}
-.glyphicon-phone:before {
-  content: "\e145";
-}
-.glyphicon-pushpin:before {
-  content: "\e146";
-}
-.glyphicon-usd:before {
-  content: "\e148";
-}
-.glyphicon-gbp:before {
-  content: "\e149";
-}
-.glyphicon-sort:before {
-  content: "\e150";
-}
-.glyphicon-sort-by-alphabet:before {
-  content: "\e151";
-}
-.glyphicon-sort-by-alphabet-alt:before {
-  content: "\e152";
-}
-.glyphicon-sort-by-order:before {
-  content: "\e153";
-}
-.glyphicon-sort-by-order-alt:before {
-  content: "\e154";
-}
-.glyphicon-sort-by-attributes:before {
-  content: "\e155";
-}
-.glyphicon-sort-by-attributes-alt:before {
-  content: "\e156";
-}
-.glyphicon-unchecked:before {
-  content: "\e157";
-}
-.glyphicon-expand:before {
-  content: "\e158";
-}
-.glyphicon-collapse-down:before {
-  content: "\e159";
-}
-.glyphicon-collapse-up:before {
-  content: "\e160";
-}
-.glyphicon-log-in:before {
-  content: "\e161";
-}
-.glyphicon-flash:before {
-  content: "\e162";
-}
-.glyphicon-log-out:before {
-  content: "\e163";
-}
-.glyphicon-new-window:before {
-  content: "\e164";
-}
-.glyphicon-record:before {
-  content: "\e165";
-}
-.glyphicon-save:before {
-  content: "\e166";
-}
-.glyphicon-open:before {
-  content: "\e167";
-}
-.glyphicon-saved:before {
-  content: "\e168";
-}
-.glyphicon-import:before {
-  content: "\e169";
-}
-.glyphicon-export:before {
-  content: "\e170";
-}
-.glyphicon-send:before {
-  content: "\e171";
-}
-.glyphicon-floppy-disk:before {
-  content: "\e172";
-}
-.glyphicon-floppy-saved:before {
-  content: "\e173";
-}
-.glyphicon-floppy-remove:before {
-  content: "\e174";
-}
-.glyphicon-floppy-save:before {
-  content: "\e175";
-}
-.glyphicon-floppy-open:before {
-  content: "\e176";
-}
-.glyphicon-credit-card:before {
-  content: "\e177";
-}
-.glyphicon-transfer:before {
-  content: "\e178";
-}
-.glyphicon-cutlery:before {
-  content: "\e179";
-}
-.glyphicon-header:before {
-  content: "\e180";
-}
-.glyphicon-compressed:before {
-  content: "\e181";
-}
-.glyphicon-earphone:before {
-  content: "\e182";
-}
-.glyphicon-phone-alt:before {
-  content: "\e183";
-}
-.glyphicon-tower:before {
-  content: "\e184";
-}
-.glyphicon-stats:before {
-  content: "\e185";
-}
-.glyphicon-sd-video:before {
-  content: "\e186";
-}
-.glyphicon-hd-video:before {
-  content: "\e187";
-}
-.glyphicon-subtitles:before {
-  content: "\e188";
-}
-.glyphicon-sound-stereo:before {
-  content: "\e189";
-}
-.glyphicon-sound-dolby:before {
-  content: "\e190";
-}
-.glyphicon-sound-5-1:before {
-  content: "\e191";
-}
-.glyphicon-sound-6-1:before {
-  content: "\e192";
-}
-.glyphicon-sound-7-1:before {
-  content: "\e193";
-}
-.glyphicon-copyright-mark:before {
-  content: "\e194";
-}
-.glyphicon-registration-mark:before {
-  content: "\e195";
-}
-.glyphicon-cloud-download:before {
-  content: "\e197";
-}
-.glyphicon-cloud-upload:before {
-  content: "\e198";
-}
-.glyphicon-tree-conifer:before {
-  content: "\e199";
-}
-.glyphicon-tree-deciduous:before {
-  content: "\e200";
-}
-.glyphicon-cd:before {
-  content: "\e201";
-}
-.glyphicon-save-file:before {
-  content: "\e202";
-}
-.glyphicon-open-file:before {
-  content: "\e203";
-}
-.glyphicon-level-up:before {
-  content: "\e204";
-}
-.glyphicon-copy:before {
-  content: "\e205";
-}
-.glyphicon-paste:before {
-  content: "\e206";
-}
-.glyphicon-alert:before {
-  content: "\e209";
-}
-.glyphicon-equalizer:before {
-  content: "\e210";
-}
-.glyphicon-king:before {
-  content: "\e211";
-}
-.glyphicon-queen:before {
-  content: "\e212";
-}
-.glyphicon-pawn:before {
-  content: "\e213";
-}
-.glyphicon-bishop:before {
-  content: "\e214";
-}
-.glyphicon-knight:before {
-  content: "\e215";
-}
-.glyphicon-baby-formula:before {
-  content: "\e216";
-}
-.glyphicon-tent:before {
-  content: "\26fa";
-}
-.glyphicon-blackboard:before {
-  content: "\e218";
-}
-.glyphicon-bed:before {
-  content: "\e219";
-}
-.glyphicon-apple:before {
-  content: "\f8ff";
-}
-.glyphicon-erase:before {
-  content: "\e221";
-}
-.glyphicon-hourglass:before {
-  content: "\231b";
-}
-.glyphicon-lamp:before {
-  content: "\e223";
-}
-.glyphicon-duplicate:before {
-  content: "\e224";
-}
-.glyphicon-piggy-bank:before {
-  content: "\e225";
-}
-.glyphicon-scissors:before {
-  content: "\e226";
-}
-.glyphicon-bitcoin:before {
-  content: "\e227";
-}
-.glyphicon-btc:before {
-  content: "\e227";
-}
-.glyphicon-xbt:before {
-  content: "\e227";
-}
-.glyphicon-yen:before {
-  content: "\00a5";
-}
-.glyphicon-jpy:before {
-  content: "\00a5";
-}
-.glyphicon-ruble:before {
-  content: "\20bd";
-}
-.glyphicon-rub:before {
-  content: "\20bd";
-}
-.glyphicon-scale:before {
-  content: "\e230";
-}
-.glyphicon-ice-lolly:before {
-  content: "\e231";
-}
-.glyphicon-ice-lolly-tasted:before {
-  content: "\e232";
-}
-.glyphicon-education:before {
-  content: "\e233";
-}
-.glyphicon-option-horizontal:before {
-  content: "\e234";
-}
-.glyphicon-option-vertical:before {
-  content: "\e235";
-}
-.glyphicon-menu-hamburger:before {
-  content: "\e236";
-}
-.glyphicon-modal-window:before {
-  content: "\e237";
-}
-.glyphicon-oil:before {
-  content: "\e238";
-}
-.glyphicon-grain:before {
-  content: "\e239";
-}
-.glyphicon-sunglasses:before {
-  content: "\e240";
-}
-.glyphicon-text-size:before {
-  content: "\e241";
-}
-.glyphicon-text-color:before {
-  content: "\e242";
-}
-.glyphicon-text-background:before {
-  content: "\e243";
-}
-.glyphicon-object-align-top:before {
-  content: "\e244";
-}
-.glyphicon-object-align-bottom:before {
-  content: "\e245";
-}
-.glyphicon-object-align-horizontal:before {
-  content: "\e246";
-}
-.glyphicon-object-align-left:before {
-  content: "\e247";
-}
-.glyphicon-object-align-vertical:before {
-  content: "\e248";
-}
-.glyphicon-object-align-right:before {
-  content: "\e249";
-}
-.glyphicon-triangle-right:before {
-  content: "\e250";
-}
-.glyphicon-triangle-left:before {
-  content: "\e251";
-}
-.glyphicon-triangle-bottom:before {
-  content: "\e252";
-}
-.glyphicon-triangle-top:before {
-  content: "\e253";
-}
-.glyphicon-console:before {
-  content: "\e254";
-}
-.glyphicon-superscript:before {
-  content: "\e255";
-}
-.glyphicon-subscript:before {
-  content: "\e256";
-}
-.glyphicon-menu-left:before {
-  content: "\e257";
-}
-.glyphicon-menu-right:before {
-  content: "\e258";
-}
-.glyphicon-menu-down:before {
-  content: "\e259";
-}
-.glyphicon-menu-up:before {
-  content: "\e260";
-}
-* {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-*:before,
-*:after {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-html {
-  font-size: 10px;
-  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
-}
-body {
-  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #333333;
-  background-color: #fff;
-}
-input,
-button,
-select,
-textarea {
-  font-family: inherit;
-  font-size: inherit;
-  line-height: inherit;
-}
-a {
-  color: #158cba;
-  text-decoration: none;
-}
-a:hover,
-a:focus {
-  color: #0d5875;
-  text-decoration: underline;
-}
-a:focus {
-  outline: thin dotted;
-  outline: 5px auto -webkit-focus-ring-color;
-  outline-offset: -2px;
-}
-figure {
-  margin: 0;
-}
-img {
-  vertical-align: middle;
-}
-.img-responsive,
-.thumbnail > img,
-.thumbnail a > img,
-.carousel-inner > .item > img,
-.carousel-inner > .item > a > img {
-  display: block;
-  max-width: 100%;
-  height: auto;
-}
-.img-rounded {
-  border-radius: 6px;
-}
-.img-thumbnail {
-  padding: 4px;
-  line-height: 1.42857143;
-  background-color: #fff;
-  border: 1px solid #ddd;
-  border-radius: 4px;
-  -webkit-transition: all 0.2s ease-in-out;
-  -o-transition: all 0.2s ease-in-out;
-  transition: all 0.2s ease-in-out;
-  display: inline-block;
-  max-width: 100%;
-  height: auto;
-}
-.img-circle {
-  border-radius: 50%;
-}
-hr {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  border: 0;
-  border-top: 1px solid #eeeeee;
-}
-.sr-only {
-  position: absolute;
-  width: 1px;
-  height: 1px;
-  margin: -1px;
-  padding: 0;
-  overflow: hidden;
-  clip: rect(0, 0, 0, 0);
-  border: 0;
-}
-.sr-only-focusable:active,
-.sr-only-focusable:focus {
-  position: static;
-  width: auto;
-  height: auto;
-  margin: 0;
-  overflow: visible;
-  clip: auto;
-}
-[role="button"] {
-  cursor: pointer;
-}
-h1,
-h2,
-h3,
-h4,
-h5,
-h6,
-.h1,
-.h2,
-.h3,
-.h4,
-.h5,
-.h6 {
-  font-family: inherit;
-  font-weight: 500;
-  line-height: 1.1;
-  color: inherit;
-}
-h1 small,
-h2 small,
-h3 small,
-h4 small,
-h5 small,
-h6 small,
-.h1 small,
-.h2 small,
-.h3 small,
-.h4 small,
-.h5 small,
-.h6 small,
-h1 .small,
-h2 .small,
-h3 .small,
-h4 .small,
-h5 .small,
-h6 .small,
-.h1 .small,
-.h2 .small,
-.h3 .small,
-.h4 .small,
-.h5 .small,
-.h6 .small {
-  font-weight: normal;
-  line-height: 1;
-  color: #777777;
-}
-h1,
-.h1,
-h2,
-.h2,
-h3,
-.h3 {
-  margin-top: 20px;
-  margin-bottom: 10px;
-}
-h1 small,
-.h1 small,
-h2 small,
-.h2 small,
-h3 small,
-.h3 small,
-h1 .small,
-.h1 .small,
-h2 .small,
-.h2 .small,
-h3 .small,
-.h3 .small {
-  font-size: 65%;
-}
-h4,
-.h4,
-h5,
-.h5,
-h6,
-.h6 {
-  margin-top: 10px;
-  margin-bottom: 10px;
-}
-h4 small,
-.h4 small,
-h5 small,
-.h5 small,
-h6 small,
-.h6 small,
-h4 .small,
-.h4 .small,
-h5 .small,
-.h5 .small,
-h6 .small,
-.h6 .small {
-  font-size: 75%;
-}
-h1,
-.h1 {
-  font-size: 28px;
-}
-h2,
-.h2 {
-  font-size: 23px;
-}
-h3,
-.h3 {
-  font-size: 21px;
-}
-h4,
-.h4 {
-  font-size: 18px;
-}
-h5,
-.h5 {
-  font-size: 14px;
-}
-h6,
-.h6 {
-  font-size: 12px;
-}
-p {
-  margin: 0 0 10px;
-}
-.lead {
-  margin-bottom: 20px;
-  font-size: 16px;
-  font-weight: 300;
-  line-height: 1.4;
-}
-@media (min-width: 768px) {
-  .lead {
-    font-size: 21px;
-  }
-}
-small,
-.small {
-  font-size: 85%;
-}
-mark,
-.mark {
-  background-color: #fcf8e3;
-  padding: .2em;
-}
-.text-left {
-  text-align: left;
-}
-.text-right {
-  text-align: right;
-}
-.text-center {
-  text-align: center;
-}
-.text-justify {
-  text-align: justify;
-}
-.text-nowrap {
-  white-space: nowrap;
-}
-.text-lowercase {
-  text-transform: lowercase;
-}
-.text-uppercase {
-  text-transform: uppercase;
-}
-.text-capitalize {
-  text-transform: capitalize;
-}
-.text-muted {
-  color: #777777;
-}
-.text-primary {
-  color: #158cba;
-}
-a.text-primary:hover,
-a.text-primary:focus {
-  color: #106a8c;
-}
-.text-success {
-  color: #3c763d;
-}
-a.text-success:hover,
-a.text-success:focus {
-  color: #2b542c;
-}
-.text-info {
-  color: #31708f;
-}
-a.text-info:hover,
-a.text-info:focus {
-  color: #245269;
-}
-.text-warning {
-  color: #8a6d3b;
-}
-a.text-warning:hover,
-a.text-warning:focus {
-  color: #66512c;
-}
-.text-danger {
-  color: #a94442;
-}
-a.text-danger:hover,
-a.text-danger:focus {
-  color: #843534;
-}
-.bg-primary {
-  color: #fff;
-  background-color: #158cba;
-}
-a.bg-primary:hover,
-a.bg-primary:focus {
-  background-color: #106a8c;
-}
-.bg-success {
-  background-color: #dff0d8;
-}
-a.bg-success:hover,
-a.bg-success:focus {
-  background-color: #c1e2b3;
-}
-.bg-info {
-  background-color: #d9edf7;
-}
-a.bg-info:hover,
-a.bg-info:focus {
-  background-color: #afd9ee;
-}
-.bg-warning {
-  background-color: #fcf8e3;
-}
-a.bg-warning:hover,
-a.bg-warning:focus {
-  background-color: #f7ecb5;
-}
-.bg-danger {
-  background-color: #f2dede;
-}
-a.bg-danger:hover,
-a.bg-danger:focus {
-  background-color: #e4b9b9;
-}
-.page-header {
-  padding-bottom: 9px;
-  margin: 40px 0 20px;
-  border-bottom: 1px solid #eeeeee;
-}
-ul,
-ol {
-  margin-top: 0;
-  margin-bottom: 10px;
-}
-ul ul,
-ol ul,
-ul ol,
-ol ol {
-  margin-bottom: 0;
-}
-.list-unstyled {
-  padding-left: 0;
-  list-style: none;
-}
-.list-inline {
-  padding-left: 0;
-  list-style: none;
-  margin-left: -5px;
-}
-.list-inline > li {
-  display: inline-block;
-  padding-left: 5px;
-  padding-right: 5px;
-}
-dl {
-  margin-top: 0;
-  margin-bottom: 20px;
-}
-dt,
-dd {
-  line-height: 1.42857143;
-}
-dt {
-  font-weight: bold;
-}
-dd {
-  margin-left: 0;
-}
-@media (min-width: 768px) {
-  .dl-horizontal dt {
-    float: left;
-    width: 160px;
-    clear: left;
-    text-align: right;
-    overflow: hidden;
-    text-overflow: ellipsis;
-    white-space: nowrap;
-  }
-  .dl-horizontal dd {
-    margin-left: 180px;
-  }
-}
-abbr[title],
-abbr[data-original-title] {
-  cursor: help;
-  border-bottom: 1px dotted #777777;
-}
-.initialism {
-  font-size: 90%;
-  text-transform: uppercase;
-}
-blockquote {
-  padding: 10px 20px;
-  margin: 0 0 20px;
-  font-size: 17.5px;
-  border-left: 5px solid #eeeeee;
-}
-blockquote p:last-child,
-blockquote ul:last-child,
-blockquote ol:last-child {
-  margin-bottom: 0;
-}
-blockquote footer,
-blockquote small,
-blockquote .small {
-  display: block;
-  font-size: 80%;
-  line-height: 1.42857143;
-  color: #777777;
-}
-blockquote footer:before,
-blockquote small:before,
-blockquote .small:before {
-  content: '\2014 \00A0';
-}
-.blockquote-reverse,
-blockquote.pull-right {
-  padding-right: 15px;
-  padding-left: 0;
-  border-right: 5px solid #eeeeee;
-  border-left: 0;
-  text-align: right;
-}
-.blockquote-reverse footer:before,
-blockquote.pull-right footer:before,
-.blockquote-reverse small:before,
-blockquote.pull-right small:before,
-.blockquote-reverse .small:before,
-blockquote.pull-right .small:before {
-  content: '';
-}
-.blockquote-reverse footer:after,
-blockquote.pull-right footer:after,
-.blockquote-reverse small:after,
-blockquote.pull-right small:after,
-.blockquote-reverse .small:after,
-blockquote.pull-right .small:after {
-  content: '\00A0 \2014';
-}
-address {
-  margin-bottom: 20px;
-  font-style: normal;
-  line-height: 1.42857143;
-}
-code,
-kbd,
-pre,
-samp {
-  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
-}
-code {
-  padding: 2px 4px;
-  font-size: 90%;
-  color: #c7254e;
-  background-color: #f9f2f4;
-  border-radius: 4px;
-}
-kbd {
-  padding: 2px 4px;
-  font-size: 90%;
-  color: #fff;
-  background-color: #333;
-  border-radius: 3px;
-  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);
-}
-kbd kbd {
-  padding: 0;
-  font-size: 100%;
-  font-weight: bold;
-  box-shadow: none;
-}
-pre {
-  display: block;
-  padding: 9.5px;
-  margin: 0 0 10px;
-  font-size: 13px;
-  line-height: 1.42857143;
-  word-break: break-all;
-  word-wrap: break-word;
-  color: #333333;
-  background-color: #f5f5f5;
-  border: 1px solid #ccc;
-  border-radius: 4px;
-}
-pre code {
-  padding: 0;
-  font-size: inherit;
-  color: inherit;
-  white-space: pre-wrap;
-  background-color: transparent;
-  border-radius: 0;
-}
-.pre-scrollable {
-  max-height: 340px;
-  overflow-y: scroll;
-}
-.container {
-  margin-right: auto;
-  margin-left: auto;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-@media (min-width: 768px) {
-  .container {
-    width: 750px;
-  }
-}
-@media (min-width: 992px) {
-  .container {
-    width: 970px;
-  }
-}
-@media (min-width: 1200px) {
-  .container {
-    width: 1170px;
-  }
-}
-.container-fluid {
-  margin-right: auto;
-  margin-left: auto;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-.row {
-  margin-left: -15px;
-  margin-right: -15px;
-}
-.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
-  position: relative;
-  min-height: 1px;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
-  float: left;
-}
-.col-xs-12 {
-  width: 100%;
-}
-.col-xs-11 {
-  width: 91.66666667%;
-}
-.col-xs-10 {
-  width: 83.33333333%;
-}
-.col-xs-9 {
-  width: 75%;
-}
-.col-xs-8 {
-  width: 66.66666667%;
-}
-.col-xs-7 {
-  width: 58.33333333%;
-}
-.col-xs-6 {
-  width: 50%;
-}
-.col-xs-5 {
-  width: 41.66666667%;
-}
-.col-xs-4 {
-  width: 33.33333333%;
-}
-.col-xs-3 {
-  width: 25%;
-}
-.col-xs-2 {
-  width: 16.66666667%;
-}
-.col-xs-1 {
-  width: 8.33333333%;
-}
-.col-xs-pull-12 {
-  right: 100%;
-}
-.col-xs-pull-11 {
-  right: 91.66666667%;
-}
-.col-xs-pull-10 {
-  right: 83.33333333%;
-}
-.col-xs-pull-9 {
-  right: 75%;
-}
-.col-xs-pull-8 {
-  right: 66.66666667%;
-}
-.col-xs-pull-7 {
-  right: 58.33333333%;
-}
-.col-xs-pull-6 {
-  right: 50%;
-}
-.col-xs-pull-5 {
-  right: 41.66666667%;
-}
-.col-xs-pull-4 {
-  right: 33.33333333%;
-}
-.col-xs-pull-3 {
-  right: 25%;
-}
-.col-xs-pull-2 {
-  right: 16.66666667%;
-}
-.col-xs-pull-1 {
-  right: 8.33333333%;
-}
-.col-xs-pull-0 {
-  right: auto;
-}
-.col-xs-push-12 {
-  left: 100%;
-}
-.col-xs-push-11 {
-  left: 91.66666667%;
-}
-.col-xs-push-10 {
-  left: 83.33333333%;
-}
-.col-xs-push-9 {
-  left: 75%;
-}
-.col-xs-push-8 {
-  left: 66.66666667%;
-}
-.col-xs-push-7 {
-  left: 58.33333333%;
-}
-.col-xs-push-6 {
-  left: 50%;
-}
-.col-xs-push-5 {
-  left: 41.66666667%;
-}
-.col-xs-push-4 {
-  left: 33.33333333%;
-}
-.col-xs-push-3 {
-  left: 25%;
-}
-.col-xs-push-2 {
-  left: 16.66666667%;
-}
-.col-xs-push-1 {
-  left: 8.33333333%;
-}
-.col-xs-push-0 {
-  left: auto;
-}
-.col-xs-offset-12 {
-  margin-left: 100%;
-}
-.col-xs-offset-11 {
-  margin-left: 91.66666667%;
-}
-.col-xs-offset-10 {
-  margin-left: 83.33333333%;
-}
-.col-xs-offset-9 {
-  margin-left: 75%;
-}
-.col-xs-offset-8 {
-  margin-left: 66.66666667%;
-}
-.col-xs-offset-7 {
-  margin-left: 58.33333333%;
-}
-.col-xs-offset-6 {
-  margin-left: 50%;
-}
-.col-xs-offset-5 {
-  margin-left: 41.66666667%;
-}
-.col-xs-offset-4 {
-  margin-left: 33.33333333%;
-}
-.col-xs-offset-3 {
-  margin-left: 25%;
-}
-.col-xs-offset-2 {
-  margin-left: 16.66666667%;
-}
-.col-xs-offset-1 {
-  margin-left: 8.33333333%;
-}
-.col-xs-offset-0 {
-  margin-left: 0%;
-}
-@media (min-width: 768px) {
-  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
-    float: left;
-  }
-  .col-sm-12 {
-    width: 100%;
-  }
-  .col-sm-11 {
-    width: 91.66666667%;
-  }
-  .col-sm-10 {
-    width: 83.33333333%;
-  }
-  .col-sm-9 {
-    width: 75%;
-  }
-  .col-sm-8 {
-    width: 66.66666667%;
-  }
-  .col-sm-7 {
-    width: 58.33333333%;
-  }
-  .col-sm-6 {
-    width: 50%;
-  }
-  .col-sm-5 {
-    width: 41.66666667%;
-  }
-  .col-sm-4 {
-    width: 33.33333333%;
-  }
-  .col-sm-3 {
-    width: 25%;
-  }
-  .col-sm-2 {
-    width: 16.66666667%;
-  }
-  .col-sm-1 {
-    width: 8.33333333%;
-  }
-  .col-sm-pull-12 {
-    right: 100%;
-  }
-  .col-sm-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-sm-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-sm-pull-9 {
-    right: 75%;
-  }
-  .col-sm-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-sm-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-sm-pull-6 {
-    right: 50%;
-  }
-  .col-sm-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-sm-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-sm-pull-3 {
-    right: 25%;
-  }
-  .col-sm-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-sm-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-sm-pull-0 {
-    right: auto;
-  }
-  .col-sm-push-12 {
-    left: 100%;
-  }
-  .col-sm-push-11 {
-    left: 91.66666667%;
-  }
-  .col-sm-push-10 {
-    left: 83.33333333%;
-  }
-  .col-sm-push-9 {
-    left: 75%;
-  }
-  .col-sm-push-8 {
-    left: 66.66666667%;
-  }
-  .col-sm-push-7 {
-    left: 58.33333333%;
-  }
-  .col-sm-push-6 {
-    left: 50%;
-  }
-  .col-sm-push-5 {
-    left: 41.66666667%;
-  }
-  .col-sm-push-4 {
-    left: 33.33333333%;
-  }
-  .col-sm-push-3 {
-    left: 25%;
-  }
-  .col-sm-push-2 {
-    left: 16.66666667%;
-  }
-  .col-sm-push-1 {
-    left: 8.33333333%;
-  }
-  .col-sm-push-0 {
-    left: auto;
-  }
-  .col-sm-offset-12 {
-    margin-left: 100%;
-  }
-  .col-sm-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-sm-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-sm-offset-9 {
-    margin-left: 75%;
-  }
-  .col-sm-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-sm-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-sm-offset-6 {
-    margin-left: 50%;
-  }
-  .col-sm-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-sm-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-sm-offset-3 {
-    margin-left: 25%;
-  }
-  .col-sm-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-sm-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-sm-offset-0 {
-    margin-left: 0%;
-  }
-}
-@media (min-width: 992px) {
-  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
-    float: left;
-  }
-  .col-md-12 {
-    width: 100%;
-  }
-  .col-md-11 {
-    width: 91.66666667%;
-  }
-  .col-md-10 {
-    width: 83.33333333%;
-  }
-  .col-md-9 {
-    width: 75%;
-  }
-  .col-md-8 {
-    width: 66.66666667%;
-  }
-  .col-md-7 {
-    width: 58.33333333%;
-  }
-  .col-md-6 {
-    width: 50%;
-  }
-  .col-md-5 {
-    width: 41.66666667%;
-  }
-  .col-md-4 {
-    width: 33.33333333%;
-  }
-  .col-md-3 {
-    width: 25%;
-  }
-  .col-md-2 {
-    width: 16.66666667%;
-  }
-  .col-md-1 {
-    width: 8.33333333%;
-  }
-  .col-md-pull-12 {
-    right: 100%;
-  }
-  .col-md-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-md-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-md-pull-9 {
-    right: 75%;
-  }
-  .col-md-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-md-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-md-pull-6 {
-    right: 50%;
-  }
-  .col-md-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-md-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-md-pull-3 {
-    right: 25%;
-  }
-  .col-md-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-md-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-md-pull-0 {
-    right: auto;
-  }
-  .col-md-push-12 {
-    left: 100%;
-  }
-  .col-md-push-11 {
-    left: 91.66666667%;
-  }
-  .col-md-push-10 {
-    left: 83.33333333%;
-  }
-  .col-md-push-9 {
-    left: 75%;
-  }
-  .col-md-push-8 {
-    left: 66.66666667%;
-  }
-  .col-md-push-7 {
-    left: 58.33333333%;
-  }
-  .col-md-push-6 {
-    left: 50%;
-  }
-  .col-md-push-5 {
-    left: 41.66666667%;
-  }
-  .col-md-push-4 {
-    left: 33.33333333%;
-  }
-  .col-md-push-3 {
-    left: 25%;
-  }
-  .col-md-push-2 {
-    left: 16.66666667%;
-  }
-  .col-md-push-1 {
-    left: 8.33333333%;
-  }
-  .col-md-push-0 {
-    left: auto;
-  }
-  .col-md-offset-12 {
-    margin-left: 100%;
-  }
-  .col-md-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-md-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-md-offset-9 {
-    margin-left: 75%;
-  }
-  .col-md-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-md-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-md-offset-6 {
-    margin-left: 50%;
-  }
-  .col-md-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-md-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-md-offset-3 {
-    margin-left: 25%;
-  }
-  .col-md-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-md-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-md-offset-0 {
-    margin-left: 0%;
-  }
-}
-@media (min-width: 1200px) {
-  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
-    float: left;
-  }
-  .col-lg-12 {
-    width: 100%;
-  }
-  .col-lg-11 {
-    width: 91.66666667%;
-  }
-  .col-lg-10 {
-    width: 83.33333333%;
-  }
-  .col-lg-9 {
-    width: 75%;
-  }
-  .col-lg-8 {
-    width: 66.66666667%;
-  }
-  .col-lg-7 {
-    width: 58.33333333%;
-  }
-  .col-lg-6 {
-    width: 50%;
-  }
-  .col-lg-5 {
-    width: 41.66666667%;
-  }
-  .col-lg-4 {
-    width: 33.33333333%;
-  }
-  .col-lg-3 {
-    width: 25%;
-  }
-  .col-lg-2 {
-    width: 16.66666667%;
-  }
-  .col-lg-1 {
-    width: 8.33333333%;
-  }
-  .col-lg-pull-12 {
-    right: 100%;
-  }
-  .col-lg-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-lg-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-lg-pull-9 {
-    right: 75%;
-  }
-  .col-lg-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-lg-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-lg-pull-6 {
-    right: 50%;
-  }
-  .col-lg-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-lg-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-lg-pull-3 {
-    right: 25%;
-  }
-  .col-lg-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-lg-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-lg-pull-0 {
-    right: auto;
-  }
-  .col-lg-push-12 {
-    left: 100%;
-  }
-  .col-lg-push-11 {
-    left: 91.66666667%;
-  }
-  .col-lg-push-10 {
-    left: 83.33333333%;
-  }
-  .col-lg-push-9 {
-    left: 75%;
-  }
-  .col-lg-push-8 {
-    left: 66.66666667%;
-  }
-  .col-lg-push-7 {
-    left: 58.33333333%;
-  }
-  .col-lg-push-6 {
-    left: 50%;
-  }
-  .col-lg-push-5 {
-    left: 41.66666667%;
-  }
-  .col-lg-push-4 {
-    left: 33.33333333%;
-  }
-  .col-lg-push-3 {
-    left: 25%;
-  }
-  .col-lg-push-2 {
-    left: 16.66666667%;
-  }
-  .col-lg-push-1 {
-    left: 8.33333333%;
-  }
-  .col-lg-push-0 {
-    left: auto;
-  }
-  .col-lg-offset-12 {
-    margin-left: 100%;
-  }
-  .col-lg-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-lg-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-lg-offset-9 {
-    margin-left: 75%;
-  }
-  .col-lg-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-lg-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-lg-offset-6 {
-    margin-left: 50%;
-  }
-  .col-lg-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-lg-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-lg-offset-3 {
-    margin-left: 25%;
-  }
-  .col-lg-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-lg-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-lg-offset-0 {
-    margin-left: 0%;
-  }
-}
-table {
-  background-color: transparent;
-}
-caption {
-  padding-top: 8px;
-  padding-bottom: 8px;
-  color: #777777;
-  text-align: left;
-}
-th {
-  text-align: left;
-}
-.table {
-  width: 100%;
-  max-width: 100%;
-  margin-bottom: 20px;
-}
-.table > thead > tr > th,
-.table > tbody > tr > th,
-.table > tfoot > tr > th,
-.table > thead > tr > td,
-.table > tbody > tr > td,
-.table > tfoot > tr > td {
-  padding: 8px;
-  line-height: 1.42857143;
-  vertical-align: top;
-  border-top: 1px solid #ddd;
-}
-.table > thead > tr > th {
-  vertical-align: bottom;
-  border-bottom: 2px solid #ddd;
-}
-.table > caption + thead > tr:first-child > th,
-.table > colgroup + thead > tr:first-child > th,
-.table > thead:first-child > tr:first-child > th,
-.table > caption + thead > tr:first-child > td,
-.table > colgroup + thead > tr:first-child > td,
-.table > thead:first-child > tr:first-child > td {
-  border-top: 0;
-}
-.table > tbody + tbody {
-  border-top: 2px solid #ddd;
-}
-.table .table {
-  background-color: #fff;
-}
-.table-condensed > thead > tr > th,
-.table-condensed > tbody > tr > th,
-.table-condensed > tfoot > tr > th,
-.table-condensed > thead > tr > td,
-.table-condensed > tbody > tr > td,
-.table-condensed > tfoot > tr > td {
-  padding: 5px;
-}
-.table-bordered {
-  border: 1px solid #ddd;
-}
-.table-bordered > thead > tr > th,
-.table-bordered > tbody > tr > th,
-.table-bordered > tfoot > tr > th,
-.table-bordered > thead > tr > td,
-.table-bordered > tbody > tr > td,
-.table-bordered > tfoot > tr > td {
-  border: 1px solid #ddd;
-}
-.table-bordered > thead > tr > th,
-.table-bordered > thead > tr > td {
-  border-bottom-width: 2px;
-}
-.table-striped > tbody > tr:nth-of-type(odd) {
-  background-color: #f9f9f9;
-}
-.table-hover > tbody > tr:hover {
-  background-color: #f5f5f5;
-}
-table col[class*="col-"] {
-  position: static;
-  float: none;
-  display: table-column;
-}
-table td[class*="col-"],
-table th[class*="col-"] {
-  position: static;
-  float: none;
-  display: table-cell;
-}
-.table > thead > tr > td.active,
-.table > tbody > tr > td.active,
-.table > tfoot > tr > td.active,
-.table > thead > tr > th.active,
-.table > tbody > tr > th.active,
-.table > tfoot > tr > th.active,
-.table > thead > tr.active > td,
-.table > tbody > tr.active > td,
-.table > tfoot > tr.active > td,
-.table > thead > tr.active > th,
-.table > tbody > tr.active > th,
-.table > tfoot > tr.active > th {
-  background-color: #f5f5f5;
-}
-.table-hover > tbody > tr > td.active:hover,
-.table-hover > tbody > tr > th.active:hover,
-.table-hover > tbody > tr.active:hover > td,
-.table-hover > tbody > tr:hover > .active,
-.table-hover > tbody > tr.active:hover > th {
-  background-color: #e8e8e8;
-}
-.table > thead > tr > td.success,
-.table > tbody > tr > td.success,
-.table > tfoot > tr > td.success,
-.table > thead > tr > th.success,
-.table > tbody > tr > th.success,
-.table > tfoot > tr > th.success,
-.table > thead > tr.success > td,
-.table > tbody > tr.success > td,
-.table > tfoot > tr.success > td,
-.table > thead > tr.success > th,
-.table > tbody > tr.success > th,
-.table > tfoot > tr.success > th {
-  background-color: #dff0d8;
-}
-.table-hover > tbody > tr > td.success:hover,
-.table-hover > tbody > tr > th.success:hover,
-.table-hover > tbody > tr.success:hover > td,
-.table-hover > tbody > tr:hover > .success,
-.table-hover > tbody > tr.success:hover > th {
-  background-color: #d0e9c6;
-}
-.table > thead > tr > td.info,
-.table > tbody > tr > td.info,
-.table > tfoot > tr > td.info,
-.table > thead > tr > th.info,
-.table > tbody > tr > th.info,
-.table > tfoot > tr > th.info,
-.table > thead > tr.info > td,
-.table > tbody > tr.info > td,
-.table > tfoot > tr.info > td,
-.table > thead > tr.info > th,
-.table > tbody > tr.info > th,
-.table > tfoot > tr.info > th {
-  background-color: #d9edf7;
-}
-.table-hover > tbody > tr > td.info:hover,
-.table-hover > tbody > tr > th.info:hover,
-.table-hover > tbody > tr.info:hover > td,
-.table-hover > tbody > tr:hover > .info,
-.table-hover > tbody > tr.info:hover > th {
-  background-color: #c4e3f3;
-}
-.table > thead > tr > td.warning,
-.table > tbody > tr > td.warning,
-.table > tfoot > tr > td.warning,
-.table > thead > tr > th.warning,
-.table > tbody > tr > th.warning,
-.table > tfoot > tr > th.warning,
-.table > thead > tr.warning > td,
-.table > tbody > tr.warning > td,
-.table > tfoot > tr.warning > td,
-.table > thead > tr.warning > th,
-.table > tbody > tr.warning > th,
-.table > tfoot > tr.warning > th {
-  background-color: #fcf8e3;
-}
-.table-hover > tbody > tr > td.warning:hover,
-.table-hover > tbody > tr > th.warning:hover,
-.table-hover > tbody > tr.warning:hover > td,
-.table-hover > tbody > tr:hover > .warning,
-.table-hover > tbody > tr.warning:hover > th {
-  background-color: #faf2cc;
-}
-.table > thead > tr > td.danger,
-.table > tbody > tr > td.danger,
-.table > tfoot > tr > td.danger,
-.table > thead > tr > th.danger,
-.table > tbody > tr > th.danger,
-.table > tfoot > tr > th.danger,
-.table > thead > tr.danger > td,
-.table > tbody > tr.danger > td,
-.table > tfoot > tr.danger > td,
-.table > thead > tr.danger > th,
-.table > tbody > tr.danger > th,
-.table > tfoot > tr.danger > th {
-  background-color: #f2dede;
-}
-.table-hover > tbody > tr > td.danger:hover,
-.table-hover > tbody > tr > th.danger:hover,
-.table-hover > tbody > tr.danger:hover > td,
-.table-hover > tbody > tr:hover > .danger,
-.table-hover > tbody > tr.danger:hover > th {
-  background-color: #ebcccc;
-}
-.table-responsive {
-  overflow-x: auto;
-  min-height: 0.01%;
-}
-@media screen and (max-width: 767px) {
-  .table-responsive {
-    width: 100%;
-    margin-bottom: 15px;
-    overflow-y: hidden;
-    -ms-overflow-style: -ms-autohiding-scrollbar;
-    border: 1px solid #ddd;
-  }
-  .table-responsive > .table {
-    margin-bottom: 0;
-  }
-  .table-responsive > .table > thead > tr > th,
-  .table-responsive > .table > tbody > tr > th,
-  .table-responsive > .table > tfoot > tr > th,
-  .table-responsive > .table > thead > tr > td,
-  .table-responsive > .table > tbody > tr > td,
-  .table-responsive > .table > tfoot > tr > td {
-    white-space: nowrap;
-  }
-  .table-responsive > .table-bordered {
-    border: 0;
-  }
-  .table-responsive > .table-bordered > thead > tr > th:first-child,
-  .table-responsive > .table-bordered > tbody > tr > th:first-child,
-  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
-  .table-responsive > .table-bordered > thead > tr > td:first-child,
-  .table-responsive > .table-bordered > tbody > tr > td:first-child,
-  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
-    border-left: 0;
-  }
-  .table-responsive > .table-bordered > thead > tr > th:last-child,
-  .table-responsive > .table-bordered > tbody > tr > th:last-child,
-  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
-  .table-responsive > .table-bordered > thead > tr > td:last-child,
-  .table-responsive > .table-bordered > tbody > tr > td:last-child,
-  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
-    border-right: 0;
-  }
-  .table-responsive > .table-bordered > tbody > tr:last-child > th,
-  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
-  .table-responsive > .table-bordered > tbody > tr:last-child > td,
-  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
-    border-bottom: 0;
-  }
-}
-fieldset {
-  padding: 0;
-  margin: 0;
-  border: 0;
-  min-width: 0;
-}
-legend {
-  display: block;
-  width: 100%;
-  padding: 0;
-  margin-bottom: 20px;
-  font-size: 21px;
-  line-height: inherit;
-  color: #333333;
-  border: 0;
-  border-bottom: 1px solid #e5e5e5;
-}
-label {
-  display: inline-block;
-  max-width: 100%;
-  margin-bottom: 5px;
-  font-weight: bold;
-}
-input[type="search"] {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-input[type="radio"],
-input[type="checkbox"] {
-  margin: 4px 0 0;
-  margin-top: 1px \9;
-  line-height: normal;
-}
-input[type="file"] {
-  display: block;
-}
-input[type="range"] {
-  display: block;
-  width: 100%;
-}
-select[multiple],
-select[size] {
-  height: auto;
-}
-input[type="file"]:focus,
-input[type="radio"]:focus,
-input[type="checkbox"]:focus {
-  outline: thin dotted;
-  outline: 5px auto -webkit-focus-ring-color;
-  outline-offset: -2px;
-}
-output {
-  display: block;
-  padding-top: 7px;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #555555;
-}
-.form-control {
-  display: block;
-  width: 100%;
-  height: 34px;
-  padding: 6px 12px;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #555555;
-  background-color: #fff;
-  background-image: none;
-  border: 1px solid #ccc;
-  border-radius: 4px;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-  -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-  transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-}
-.form-control:focus {
-  border-color: #66afe9;
-  outline: 0;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
-  box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
-}
-.form-control::-moz-placeholder {
-  color: #999;
-  opacity: 1;
-}
-.form-control:-ms-input-placeholder {
-  color: #999;
-}
-.form-control::-webkit-input-placeholder {
-  color: #999;
-}
-.form-control[disabled],
-.form-control[readonly],
-fieldset[disabled] .form-control {
-  background-color: #eeeeee;
-  opacity: 1;
-}
-.form-control[disabled],
-fieldset[disabled] .form-control {
-  cursor: not-allowed;
-}
-textarea.form-control {
-  height: auto;
-}
-input[type="search"] {
-  -webkit-appearance: none;
-}
-@media screen and (-webkit-min-device-pixel-ratio: 0) {
-  input[type="date"].form-control,
-  input[type="time"].form-control,
-  input[type="datetime-local"].form-control,
-  input[type="month"].form-control {
-    line-height: 34px;
-  }
-  input[type="date"].input-sm,
-  input[type="time"].input-sm,
-  input[type="datetime-local"].input-sm,
-  input[type="month"].input-sm,
-  .input-group-sm input[type="date"],
-  .input-group-sm input[type="time"],
-  .input-group-sm input[type="datetime-local"],
-  .input-group-sm input[type="month"] {
-    line-height: 30px;
-  }
-  input[type="date"].input-lg,
-  input[type="time"].input-lg,
-  input[type="datetime-local"].input-lg,
-  input[type="month"].input-lg,
-  .input-group-lg input[type="date"],
-  .input-group-lg input[type="time"],
-  .input-group-lg input[type="datetime-local"],
-  .input-group-lg input[type="month"] {
-    line-height: 46px;
-  }
-}
-.form-group {
-  margin-bottom: 15px;
-}
-.radio,
-.checkbox {
-  position: relative;
-  display: block;
-  margin-top: 10px;
-  margin-bottom: 10px;
-}
-.radio label,
-.checkbox label {
-  min-height: 20px;
-  padding-left: 20px;
-  margin-bottom: 0;
-  font-weight: normal;
-  cursor: pointer;
-}
-.radio input[type="radio"],
-.radio-inline input[type="radio"],
-.checkbox input[type="checkbox"],
-.checkbox-inline input[type="checkbox"] {
-  position: absolute;
-  margin-left: -20px;
-  margin-top: 4px \9;
-}
-.radio + .radio,
-.checkbox + .checkbox {
-  margin-top: -5px;
-}
-.radio-inline,
-.checkbox-inline {
-  position: relative;
-  display: inline-block;
-  padding-left: 20px;
-  margin-bottom: 0;
-  vertical-align: middle;
-  font-weight: normal;
-  cursor: pointer;
-}
-.radio-inline + .radio-inline,
-.checkbox-inline + .checkbox-inline {
-  margin-top: 0;
-  margin-left: 10px;
-}
-input[type="radio"][disabled],
-input[type="checkbox"][disabled],
-input[type="radio"].disabled,
-input[type="checkbox"].disabled,
-fieldset[disabled] input[type="radio"],
-fieldset[disabled] input[type="checkbox"] {
-  cursor: not-allowed;
-}
-.radio-inline.disabled,
-.checkbox-inline.disabled,
-fieldset[disabled] .radio-inline,
-fieldset[disabled] .checkbox-inline {
-  cursor: not-allowed;
-}
-.radio.disabled label,
-.checkbox.disabled label,
-fieldset[disabled] .radio label,
-fieldset[disabled] .checkbox label {
-  cursor: not-allowed;
-}
-.form-control-static {
-  padding-top: 7px;
-  padding-bottom: 7px;
-  margin-bottom: 0;
-  min-height: 34px;
-}
-.form-control-static.input-lg,
-.form-control-static.input-sm {
-  padding-left: 0;
-  padding-right: 0;
-}
-.input-sm {
-  height: 30px;
-  padding: 5px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-  border-radius: 3px;
-}
-select.input-sm {
-  height: 30px;
-  line-height: 30px;
-}
-textarea.input-sm,
-select[multiple].input-sm {
-  height: auto;
-}
-.form-group-sm .form-control {
-  height: 30px;
-  padding: 5px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-  border-radius: 3px;
-}
-.form-group-sm select.form-control {
-  height: 30px;
-  line-height: 30px;
-}
-.form-group-sm textarea.form-control,
-.form-group-sm select[multiple].form-control {
-  height: auto;
-}
-.form-group-sm .form-control-static {
-  height: 30px;
-  min-height: 32px;
-  padding: 6px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-}
-.input-lg {
-  height: 46px;
-  padding: 10px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-  border-radius: 6px;
-}
-select.input-lg {
-  height: 46px;
-  line-height: 46px;
-}
-textarea.input-lg,
-select[multiple].input-lg {
-  height: auto;
-}
-.form-group-lg .form-control {
-  height: 46px;
-  padding: 10px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-  border-radius: 6px;
-}
-.form-group-lg select.form-control {
-  height: 46px;
-  line-height: 46px;
-}
-.form-group-lg textarea.form-control,
-.form-group-lg select[multiple].form-control {
-  height: auto;
-}
-.form-group-lg .form-control-static {
-  height: 46px;
-  min-height: 38px;
-  padding: 11px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-}
-.has-feedback {
-  position: relative;
-}
-.has-feedback .form-control {
-  padding-right: 42.5px;
-}
-.form-control-feedback {
-  position: absolute;
-  top: 0;
-  right: 0;
-  z-index: 2;
-  display: block;
-  width: 34px;
-  height: 34px;
-  line-height: 34px;
-  text-align: center;
-  pointer-events: none;
-}
-.input-lg + .form-control-feedback,
-.input-group-lg + .form-control-feedback,
-.form-group-lg .form-control + .form-control-feedback {
-  width: 46px;
-  height: 46px;
-  line-height: 46px;
-}
-.input-sm + .form-control-feedback,
-.input-group-sm + .form-control-feedback,
-.form-group-sm .form-control + .form-control-feedback {
-  width: 30px;
-  height: 30px;
-  line-height: 30px;
-}
-.has-success .help-block,
-.has-success .control-label,
-.has-success .radio,
-.has-success .checkbox,
-.has-success .radio-inline,
-.has-success .checkbox-inline,
-.has-success.radio label,
-.has-success.checkbox label,
-.has-success.radio-inline label,
-.has-success.checkbox-inline label {
-  color: #3c763d;
-}
-.has-success .form-control {
-  border-color: #3c763d;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-success .form-control:focus {
-  border-color: #2b542c;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
-}
-.has-success .input-group-addon {
-  color: #3c763d;
-  border-color: #3c763d;
-  background-color: #dff0d8;
-}
-.has-success .form-control-feedback {
-  color: #3c763d;
-}
-.has-warning .help-block,
-.has-warning .control-label,
-.has-warning .radio,
-.has-warning .checkbox,
-.has-warning .radio-inline,
-.has-warning .checkbox-inline,
-.has-warning.radio label,
-.has-warning.checkbox label,
-.has-warning.radio-inline label,
-.has-warning.checkbox-inline label {
-  color: #8a6d3b;
-}
-.has-warning .form-control {
-  border-color: #8a6d3b;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-warning .form-control:focus {
-  border-color: #66512c;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
-}
-.has-warning .input-group-addon {
-  color: #8a6d3b;
-  border-color: #8a6d3b;
-  background-color: #fcf8e3;
-}
-.has-warning .form-control-feedback {
-  color: #8a6d3b;
-}
-.has-error .help-block,
-.has-error .control-label,
-.has-error .radio,
-.has-error .checkbox,
-.has-error .radio-inline,
-.has-error .checkbox-inline,
-.has-error.radio label,
-.has-error.checkbox label,
-.has-error.radio-inline label,
-.has-error.checkbox-inline label {
-  color: #a94442;
-}
-.has-error .form-control {
-  border-color: #a94442;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-error .form-control:focus {
-  border-color: #843534;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
-}
-.has-error .input-group-addon {
-  color: #a94442;
-  border-color: #a94442;
-  background-color: #f2dede;
-}
-.has-error .form-control-feedback {
-  color: #a94442;
-}
-.has-feedback label ~ .form-control-feedback {
-  top: 25px;
-}
-.has-feedback label.sr-only ~ .form-control-feedback {
-  top: 0;
-}
-.help-block {
-  display: block;
-  margin-top: 5px;
-  margin-bottom: 10px;
-  color: #737373;
-}
-@media (min-width: 768px) {
-  .form-inline .form-group {
-    display: inline-block;
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .form-control {
-    display: inline-block;
-    width: auto;
-    vertical-align: middle;
-  }
-  .form-inline .form-control-static {
-    display: inline-block;
-  }
-  .form-inline .input-group {
-    display: inline-table;
-    vertical-align: middle;
-  }
-  .form-inline .input-group .input-group-addon,
-  .form-inline .input-group .input-group-btn,
-  .form-inline .input-group .form-control {
-    width: auto;
-  }
-  .form-inline .input-group > .form-control {
-    width: 100%;
-  }
-  .form-inline .control-label {
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .radio,
-  .form-inline .checkbox {
-    display: inline-block;
-    margin-top: 0;
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .radio label,
-  .form-inline .checkbox label {
-    padding-left: 0;
-  }
-  .form-inline .radio input[type="radio"],
-  .form-inline .checkbox input[type="checkbox"] {
-    position: relative;
-    margin-left: 0;
-  }
-  .form-inline .has-feedback .form-control-feedback {
-    top: 0;
-  }
-}
-.form-horizontal .radio,
-.form-horizontal .checkbox,
-.form-horizontal .radio-inline,
-.form-horizontal .checkbox-inline {
-  margin-top: 0;
-  margin-bottom: 0;
-  padding-top: 7px;
-}
-.form-horizontal .radio,
-.form-horizontal .checkbox {
-  min-height: 27px;
-}
-.form-horizontal .form-group {
-  margin-left: -15px;
-  margin-right: -15px;
-}
-@media (min-width: 768px) {
-  .form-horizontal .control-label {
-    text-align: right;
-    margin-bottom: 0;
-    padding-top: 7px;
-  }
-}
-.form-horizontal .has-feedback .form-control-feedback {
-  right: 15px;
-}
-@media (min-width: 768px) {
-  .form-horizontal .form-group-lg .control-label {
-    padding-top: 14.333333px;
-    font-size: 18px;
-  }
-}
-@media (min-width: 768px) {
-  .form-horizontal .form-group-sm .control-label {
-    padding-top: 6px;
-    font-size: 12px;
-  }
-}
-.btn {
-  display: inline-block;
-  margin-bottom: 0;
-  font-weight: normal;
-  text-align: center;
-  vertical-align: middle;
-  touch-action: manipulation;
-  cursor: pointer;
-  background-image: none;
-  border: 1px s

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf b/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf
deleted file mode 100644
index f7936cc..0000000
Binary files a/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf and /dev/null differ


[35/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/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
new file mode 100644
index 0000000..d370765
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/js/index.js
@@ -0,0 +1,1420 @@
+angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
+  $rootScope.sidebarVisible = false;
+  return $rootScope.showSidebar = function() {
+    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
+    return $rootScope.sidebarClass = 'force-show';
+  };
+}]).value('flinkConfig', {
+  "refresh-interval": 10000
+}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
+  MainService.loadConfig().then(function(config) {
+    angular.extend(flinkConfig, config);
+    JobsService.listJobs();
+    return $interval(function() {
+      return JobsService.listJobs();
+    }, flinkConfig["refresh-interval"]);
+  });
+  Highcharts.setOptions({
+    global: {
+      useUTC: false
+    }
+  });
+  Highcharts.createElement('link', {
+    href: '//fonts.googleapis.com/css?family=Dosis:400,600',
+    rel: 'stylesheet',
+    type: 'text/css'
+  }, null, document.getElementsByTagName('head')[0]);
+  Highcharts.theme = {
+    colors: ["#7cb5ec", "#f7a35c", "#90ee7e", "#7798BF", "#aaeeee", "#ff0066", "#eeaaee", "#55BF3B", "#DF5353", "#7798BF", "#aaeeee"],
+    chart: {
+      backgroundColor: null,
+      style: {
+        fontFamily: "Dosis, sans-serif"
+      }
+    },
+    title: {
+      style: {
+        fontSize: '16px',
+        fontWeight: 'bold',
+        textTransform: 'uppercase'
+      }
+    },
+    tooltip: {
+      borderWidth: 0,
+      backgroundColor: 'rgba(219,219,216,0.8)',
+      shadow: false
+    },
+    legend: {
+      itemStyle: {
+        fontWeight: 'bold',
+        fontSize: '13px'
+      }
+    },
+    xAxis: {
+      gridLineWidth: 1,
+      labels: {
+        style: {
+          fontSize: '12px'
+        }
+      }
+    },
+    yAxis: {
+      minorTickInterval: 'auto',
+      title: {
+        style: {
+          textTransform: 'uppercase'
+        }
+      },
+      labels: {
+        style: {
+          fontSize: '12px'
+        }
+      }
+    },
+    plotOptions: {
+      candlestick: {
+        lineColor: '#404048'
+      }
+    },
+    background2: '#F0F0EA'
+  };
+  return Highcharts.setOptions(Highcharts.theme);
+}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
+  return $uiViewScrollProvider.useAnchorScroll();
+}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
+  $stateProvider.state("overview", {
+    url: "/overview",
+    views: {
+      main: {
+        templateUrl: "partials/overview.html",
+        controller: 'OverviewController'
+      }
+    }
+  }).state("running-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: true,
+    views: {
+      main: {
+        templateUrl: "partials/jobs/job.html",
+        controller: 'SingleJobController'
+      }
+    }
+  }).state("single-job.plan", {
+    url: "",
+    abstract: true,
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.plan.html",
+        controller: 'JobPlanController'
+      }
+    }
+  }).state("single-job.plan.overview", {
+    url: "",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
+        controller: 'JobPlanOverviewController'
+      }
+    }
+  }).state("single-job.plan.accumulators", {
+    url: "/accumulators",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
+        controller: 'JobPlanAccumulatorsController'
+      }
+    }
+  }).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.statistics", {
+    url: "/statistics",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.statistics.html"
+      }
+    }
+  }).state("single-job.exceptions", {
+    url: "/exceptions",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.exceptions.html",
+        controller: 'JobExceptionsController'
+      }
+    }
+  }).state("single-job.properties", {
+    url: "/properties",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.properties.html",
+        controller: 'JobPropertiesController'
+      }
+    }
+  }).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}",
+    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.log", {
+    url: "/logfile",
+    views: {
+      details: {
+        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
+      }
+    }
+  }).state("single-manager.stdout", {
+    url: "/stdout",
+    views: {
+      details: {
+        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
+      }
+    }
+  }).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'
+      }
+    }
+  });
+  return $urlRouterProvider.otherwise("/overview");
+}]);
+
+angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
+  return {
+    transclude: true,
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: "@"
+    },
+    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'label label-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
+  return {
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: '@'
+    },
+    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('tableProperty', function() {
+  return {
+    replace: true,
+    scope: {
+      value: '='
+    },
+    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
+  };
+});
+
+angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
+  var amDurationFormatExtendedFilter;
+  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
+    if (typeof value === "undefined" || value === null) {
+      return "";
+    }
+    return moment.duration(value, format).format(durationFormat, {
+      trim: false
+    });
+  };
+  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
+  return amDurationFormatExtendedFilter;
+}]).filter("humanizeText", function() {
+  return function(text) {
+    if (text) {
+      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
+    } else {
+      return '';
+    }
+  };
+}).filter("bytes", function() {
+  return function(bytes, precision) {
+    var number, units;
+    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
+      return "-";
+    }
+    if (typeof precision === "undefined") {
+      precision = 1;
+    }
+    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
+    number = Math.floor(Math.log(bytes) / Math.log(1024));
+    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
+  };
+});
+
+angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("config").success(function(data, status, headers, config) {
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
+  return JobManagerConfigService.loadConfig().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['config'] = data;
+  });
+}]).controller('JobManagerLogsController', ["$scope", "JobManagerLogsService", function($scope, JobManagerLogsService) {
+  JobManagerLogsService.loadLogs().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['log'] = data;
+  });
+  return $scope.reloadData = function() {
+    return JobManagerLogsService.loadLogs().then(function(data) {
+      return $scope.jobmanager['log'] = data;
+    });
+  };
+}]).controller('JobManagerStdoutController', ["$scope", "JobManagerStdoutService", function($scope, JobManagerStdoutService) {
+  JobManagerStdoutService.loadStdout().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['stdout'] = data;
+  });
+  return $scope.reloadData = function() {
+    return JobManagerStdoutService.loadStdout().then(function(data) {
+      return $scope.jobmanager['stdout'] = data;
+    });
+  };
+}]);
+
+angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var config;
+  config = {};
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/config").success(function(data, status, headers, config) {
+      config = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('JobManagerLogsService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var logs;
+  logs = {};
+  this.loadLogs = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/log").success(function(data, status, headers, config) {
+      logs = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('JobManagerStdoutService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var stdout;
+  stdout = {};
+  this.loadStdout = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/stdout").success(function(data, status, headers, config) {
+      stdout = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('running');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
+  var refresher;
+  console.log('SingleJobController');
+  $scope.jobid = $stateParams.jobid;
+  $scope.job = null;
+  $scope.plan = null;
+  $scope.vertices = null;
+  JobsService.loadJob($stateParams.jobid).then(function(data) {
+    $scope.job = data;
+    $scope.plan = data.plan;
+    return $scope.vertices = data.vertices;
+  });
+  refresher = $interval(function() {
+    return JobsService.loadJob($stateParams.jobid).then(function(data) {
+      $scope.job = data;
+      return $scope.$broadcast('reload');
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    $scope.job = null;
+    $scope.plan = null;
+    $scope.vertices = null;
+    return $interval.cancel(refresher);
+  });
+}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobPlanController');
+  $scope.nodeid = null;
+  $scope.nodeUnfolded = false;
+  $scope.stateList = JobsService.stateList();
+  $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      $scope.accumulators = null;
+      return $scope.$broadcast('reload');
+    } else {
+      $scope.nodeid = null;
+      $scope.nodeUnfolded = false;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      return $scope.accumulators = null;
+    }
+  };
+  $scope.deactivateNode = function() {
+    $scope.nodeid = null;
+    $scope.nodeUnfolded = false;
+    $scope.vertex = null;
+    $scope.subtasks = null;
+    return $scope.accumulators = null;
+  };
+  return $scope.toggleFold = function() {
+    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
+  };
+}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanOverviewController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
+    JobsService.getSubtasks($scope.nodeid).then(function(data) {
+      return $scope.subtasks = data;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanOverviewController');
+    if ($scope.nodeid) {
+      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
+        return $scope.subtasks = data;
+      });
+    }
+  });
+}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanAccumulatorsController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
+    JobsService.getAccumulators($scope.nodeid).then(function(data) {
+      $scope.accumulators = data.main;
+      return $scope.subtaskAccumulators = data.subtasks;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanAccumulatorsController');
+    if ($scope.nodeid) {
+      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
+        $scope.accumulators = data.main;
+        return $scope.subtaskAccumulators = data.subtasks;
+      });
+    }
+  });
+}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobTimelineVertexController');
+  JobsService.getVertex($stateParams.vertexId).then(function(data) {
+    return $scope.vertex = data;
+  });
+  return $scope.$on('reload', function(event) {
+    console.log('JobTimelineVertexController');
+    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
+      return $scope.vertex = data;
+    });
+  });
+}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  return JobsService.loadExceptions().then(function(data) {
+    return $scope.exceptions = data;
+  });
+}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPropertiesController');
+  return $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      return JobsService.getNode(nodeid).then(function(data) {
+        return $scope.node = data;
+      });
+    } else {
+      $scope.nodeid = null;
+      return $scope.node = null;
+    }
+  };
+}]);
+
+angular.module('flinkApp').directive('vertex', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
+    scope: {
+      data: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data.subtasks, function(subtask, i) {
+          var times;
+          times = [
+            {
+              label: "Scheduled",
+              color: "#666",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["SCHEDULED"],
+              ending_time: subtask.timestamps["DEPLOYING"],
+              type: 'regular'
+            }, {
+              label: "Deploying",
+              color: "#aaa",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["DEPLOYING"],
+              ending_time: subtask.timestamps["RUNNING"],
+              type: 'regular'
+            }
+          ];
+          if (subtask.timestamps["FINISHED"] > 0) {
+            times.push({
+              label: "Running",
+              color: "#ddd",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["RUNNING"],
+              ending_time: subtask.timestamps["FINISHED"],
+              type: 'regular'
+            });
+          }
+          return testData.push({
+            label: "(" + subtask.subtask + ") " + subtask.host,
+            times: times
+          });
+        });
+        chart = d3.timeline().stack().tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("single").labelFormat(function(label) {
+          return label;
+        }).margin({
+          left: 100,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).relativeTime();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      analyzeTime(scope.data);
+    }
+  };
+}]).directive('timeline', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline' width='0' height='0'></svg>",
+    scope: {
+      vertices: "=",
+      jobid: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl, translateLabel;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      translateLabel = function(label) {
+        return label.replace("&gt;", ">");
+      };
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data, function(vertex) {
+          if (vertex['start-time'] > -1) {
+            if (vertex.type === 'scheduled') {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#cccccc",
+                    borderColor: "#555555",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    type: vertex.type
+                  }
+                ]
+              });
+            } else {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#d9f1f7",
+                    borderColor: "#62cdea",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    link: vertex.id,
+                    type: vertex.type
+                  }
+                ]
+              });
+            }
+          }
+        });
+        chart = d3.timeline().stack().click(function(d, i, datum) {
+          if (d.link) {
+            return $state.go("single-job.timeline.vertex", {
+              jobid: scope.jobid,
+              vertexId: d.link
+            });
+          }
+        }).tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("main").margin({
+          left: 0,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).showBorderLine().showHourTimeline();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      scope.$watch(attrs.vertices, function(data) {
+        if (data) {
+          return analyzeTime(data);
+        }
+      });
+    }
+  };
+}]).directive('jobPlan', ["$timeout", function($timeout) {
+  return {
+    template: "<svg class='graph' width='500' height='400'><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: '=',
+      setNode: '&'
+    },
+    link: function(scope, elem, attrs) {
+      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
+      g = null;
+      mainZoom = d3.behavior.zoom();
+      subgraphs = [];
+      jobid = attrs.jobid;
+      mainSvgElement = elem.children()[0];
+      mainG = elem.children().children()[0];
+      mainTmpElement = elem.children()[1];
+      d3mainSvg = d3.select(mainSvgElement);
+      d3mainSvgG = d3.select(mainG);
+      d3tmpSvg = d3.select(mainTmpElement);
+      containerW = elem.width();
+      angular.element(elem.children()[0]).width(containerW);
+      scope.zoomIn = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() < 2.99) {
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          mainZoom.scale(mainZoom.scale() + 0.1);
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      scope.zoomOut = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() > 0.31) {
+          mainZoom.scale(mainZoom.scale() - 0.1);
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      createLabelEdge = function(el) {
+        var labelValue;
+        labelValue = "";
+        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
+          labelValue += "<div class='edge-label'>";
+          if (el.ship_strategy != null) {
+            labelValue += el.ship_strategy;
+          }
+          if (el.temp_mode !== undefined) {
+            labelValue += " (" + el.temp_mode + ")";
+          }
+          if (el.local_strategy !== undefined) {
+            labelValue += ",<br>" + el.local_strategy;
+          }
+          labelValue += "</div>";
+        }
+        return labelValue;
+      };
+      isSpecialIterationNode = function(info) {
+        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
+      };
+      getNodeType = function(el, info) {
+        if (info === "mirror") {
+          return 'node-mirror';
+        } else if (isSpecialIterationNode(info)) {
+          return 'node-iteration';
+        } else {
+          return 'node-normal';
+        }
+      };
+      createLabelNode = function(el, info, maxW, maxH) {
+        var labelValue, stepName;
+        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
+        if (info === "mirror") {
+          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
+        } else {
+          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
+        }
+        if (el.description === "") {
+          labelValue += "";
+        } else {
+          stepName = el.description;
+          stepName = shortenString(stepName);
+          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
+        }
+        if (el.step_function != null) {
+          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
+        } else {
+          if (isSpecialIterationNode(info)) {
+            labelValue += "<h5>" + info + " Node</h5>";
+          }
+          if (el.parallelism !== "") {
+            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
+          }
+          if (el.operator !== undefined) {
+            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
+          }
+        }
+        labelValue += "</div>";
+        return labelValue;
+      };
+      extendLabelNodeForIteration = function(id, maxW, maxH) {
+        var labelValue, svgID;
+        svgID = "svg-" + id;
+        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
+        return labelValue;
+      };
+      shortenString = function(s) {
+        var sbr;
+        if (s.charAt(0) === "<") {
+          s = s.replace("<", "&lt;");
+          s = s.replace(">", "&gt;");
+        }
+        sbr = "";
+        while (s.length > 30) {
+          sbr = sbr + s.substring(0, 30) + "<br>";
+          s = s.substring(30, s.length);
+        }
+        sbr = sbr + s;
+        return sbr;
+      };
+      createNode = function(g, data, el, isParent, maxW, maxH) {
+        if (isParent == null) {
+          isParent = false;
+        }
+        if (el.id === data.partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "partialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "partialSolution")
+          });
+        } else if (el.id === data.next_partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextPartialSolution")
+          });
+        } else if (el.id === data.workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "workset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "workset")
+          });
+        } else if (el.id === data.next_workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextWorkset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextWorkset")
+          });
+        } else if (el.id === data.solution_set) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionSet", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionSet")
+          });
+        } else if (el.id === data.solution_delta) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionDelta", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionDelta")
+          });
+        } else {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "")
+          });
+        }
+      };
+      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
+        var missingNode;
+        if (existingNodes.indexOf(pred.id) !== -1) {
+          return g.setEdge(pred.id, el.id, {
+            label: createLabelEdge(pred),
+            labelType: 'html',
+            arrowhead: 'normal'
+          });
+        } else {
+          missingNode = searchForNode(data, pred.id);
+          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
+            missingNodes.push(missingNode.id);
+            g.setNode(missingNode.id, {
+              label: createLabelNode(missingNode, "mirror"),
+              labelType: 'html',
+              "class": getNodeType(missingNode, 'mirror')
+            });
+            return g.setEdge(missingNode.id, el.id, {
+              label: createLabelEdge(missingNode),
+              labelType: 'html'
+            });
+          }
+        }
+      };
+      loadJsonToDagre = function(g, data) {
+        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
+        existingNodes = [];
+        missingNodes = [];
+        if (data.nodes != null) {
+          toIterate = data.nodes;
+        } else {
+          toIterate = data.step_function;
+          isParent = true;
+        }
+        for (k = 0, len = toIterate.length; k < len; k++) {
+          el = toIterate[k];
+          maxW = 0;
+          maxH = 0;
+          if (el.step_function) {
+            sg = new dagreD3.graphlib.Graph({
+              multigraph: true,
+              compound: true
+            }).setGraph({
+              nodesep: 20,
+              edgesep: 0,
+              ranksep: 20,
+              rankdir: "LR",
+              marginx: 10,
+              marginy: 10
+            });
+            subgraphs[el.id] = sg;
+            loadJsonToDagre(sg, el);
+            r = new dagreD3.render();
+            d3tmpSvg.select('g').call(r, sg);
+            maxW = sg.graph().width;
+            maxH = sg.graph().height;
+            angular.element(mainTmpElement).empty();
+          }
+          createNode(g, data, el, isParent, maxW, maxH);
+          existingNodes.push(el.id);
+          if (el.inputs != null) {
+            ref = el.inputs;
+            for (l = 0, len1 = ref.length; l < len1; l++) {
+              pred = ref[l];
+              createEdge(g, data, el, existingNodes, pred, missingNodes);
+            }
+          }
+        }
+        return g;
+      };
+      searchForNode = function(data, nodeID) {
+        var el, i, j;
+        for (i in data.nodes) {
+          el = data.nodes[i];
+          if (el.id === nodeID) {
+            return el;
+          }
+          if (el.step_function != null) {
+            for (j in el.step_function) {
+              if (el.step_function[j].id === nodeID) {
+                return el.step_function[j];
+              }
+            }
+          }
+        }
+      };
+      drawGraph = function(data) {
+        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
+        g = new dagreD3.graphlib.Graph({
+          multigraph: true,
+          compound: true
+        }).setGraph({
+          nodesep: 70,
+          edgesep: 0,
+          ranksep: 50,
+          rankdir: "LR",
+          marginx: 40,
+          marginy: 40
+        });
+        loadJsonToDagre(g, data);
+        renderer = new dagreD3.render();
+        d3mainSvgG.call(renderer, g);
+        for (i in subgraphs) {
+          sg = subgraphs[i];
+          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
+        }
+        newScale = 0.5;
+        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
+        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
+        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
+        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
+        mainZoom.on("zoom", function() {
+          var ev;
+          ev = d3.event;
+          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
+        });
+        mainZoom(d3mainSvg);
+        return d3mainSvgG.selectAll('.node').on('click', function(d) {
+          return scope.setNode({
+            nodeid: d
+          });
+        });
+      };
+      scope.$watch(attrs.plan, function(newPlan) {
+        if (newPlan) {
+          return drawGraph(newPlan);
+        }
+      });
+    }
+  };
+}]);
+
+angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
+  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
+  currentJob = null;
+  currentPlan = null;
+  deferreds = {};
+  jobs = {
+    running: [],
+    finished: [],
+    cancelled: [],
+    failed: []
+  };
+  jobObservers = [];
+  notifyObservers = function() {
+    return angular.forEach(jobObservers, function(callback) {
+      return callback();
+    });
+  };
+  this.registerObserver = function(callback) {
+    return jobObservers.push(callback);
+  };
+  this.unRegisterObserver = function(callback) {
+    var index;
+    index = jobObservers.indexOf(callback);
+    return jobObservers.splice(index, 1);
+  };
+  this.stateList = function() {
+    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
+  };
+  this.translateLabelState = function(state) {
+    switch (state.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(list) {
+    return angular.forEach(list, function(item, jobKey) {
+      if (!(item['end-time'] > -1)) {
+        return item['end-time'] = item['start-time'] + item['duration'];
+      }
+    });
+  };
+  this.processVertices = function(data) {
+    angular.forEach(data.vertices, function(vertex, i) {
+      return vertex.type = 'regular';
+    });
+    return data.vertices.unshift({
+      name: 'Scheduled',
+      'start-time': data.timestamps['CREATED'],
+      'end-time': data.timestamps['CREATED'] + 1,
+      type: 'scheduled'
+    });
+  };
+  this.listJobs = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("joboverview").success((function(_this) {
+      return function(data, status, headers, config) {
+        angular.forEach(data, function(list, listKey) {
+          switch (listKey) {
+            case 'running':
+              return jobs.running = _this.setEndTimes(list);
+            case 'finished':
+              return jobs.finished = _this.setEndTimes(list);
+            case 'cancelled':
+              return jobs.cancelled = _this.setEndTimes(list);
+            case 'failed':
+              return jobs.failed = _this.setEndTimes(list);
+          }
+        });
+        deferred.resolve(jobs);
+        return notifyObservers();
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getJobs = function(type) {
+    return jobs[type];
+  };
+  this.getAllJobs = function() {
+    return jobs;
+  };
+  this.loadJob = function(jobid) {
+    currentJob = null;
+    deferreds.job = $q.defer();
+    $http.get("jobs/" + jobid).success((function(_this) {
+      return function(data, status, headers, config) {
+        _this.setEndTimes(data.vertices);
+        _this.processVertices(data);
+        return $http.get("jobs/" + jobid + "/config").success(function(jobConfig) {
+          data = angular.extend(data, jobConfig);
+          currentJob = data;
+          return deferreds.job.resolve(currentJob);
+        });
+      };
+    })(this));
+    return deferreds.job.promise;
+  };
+  this.getNode = function(nodeid) {
+    var deferred, seekNode;
+    seekNode = function(nodeid, data) {
+      var j, len, node, sub;
+      for (j = 0, len = data.length; j < len; j++) {
+        node = data[j];
+        if (node.id === nodeid) {
+          return node;
+        }
+        if (node.step_function) {
+          sub = seekNode(nodeid, node.step_function);
+        }
+        if (sub) {
+          return sub;
+        }
+      }
+      return null;
+    };
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var foundNode;
+        foundNode = seekNode(nodeid, currentJob.plan.nodes);
+        foundNode.vertex = _this.seekVertex(nodeid);
+        return deferred.resolve(foundNode);
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.seekVertex = function(nodeid) {
+    var j, len, ref, vertex;
+    ref = currentJob.vertices;
+    for (j = 0, len = ref.length; j < len; j++) {
+      vertex = ref[j];
+      if (vertex.id === nodeid) {
+        return vertex;
+      }
+    }
+    return null;
+  };
+  this.getVertex = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var vertex;
+        vertex = _this.seekVertex(vertexid);
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
+          vertex.subtasks = data.subtasks;
+          return deferred.resolve(vertex);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getSubtasks = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
+          var subtasks;
+          subtasks = data.subtasks;
+          return deferred.resolve(subtasks);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getAccumulators = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
+          var accumulators;
+          accumulators = data['user-accumulators'];
+          return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
+            var subtaskAccumulators;
+            subtaskAccumulators = data.subtasks;
+            return deferred.resolve({
+              main: accumulators,
+              subtasks: subtaskAccumulators
+            });
+          });
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.loadExceptions = function() {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
+          currentJob.exceptions = exceptions;
+          return deferred.resolve(exceptions);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
+  var refresh;
+  $scope.jobObserver = function() {
+    $scope.runningJobs = JobsService.getJobs('running');
+    return $scope.finishedJobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  $scope.jobObserver();
+  OverviewService.loadOverview().then(function(data) {
+    return $scope.overview = data;
+  });
+  refresh = $interval(function() {
+    return OverviewService.loadOverview().then(function(data) {
+      return $scope.overview = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]);
+
+angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var overview;
+  overview = {};
+  this.loadOverview = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("overview").success(function(data, status, headers, config) {
+      overview = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('AllTaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
+  var refresh;
+  TaskManagersService.loadManagers().then(function(data) {
+    return $scope.managers = data;
+  });
+  refresh = $interval(function() {
+    return TaskManagersService.loadManagers().then(function(data) {
+      return $scope.managers = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]).controller('SingleTaskManagerController', ["$scope", "$stateParams", "SingleTaskManagerService", "$interval", "flinkConfig", function($scope, $stateParams, SingleTaskManagerService, $interval, flinkConfig) {
+  var refresh;
+  $scope.metrics = {};
+  SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
+    return $scope.metrics = data[0];
+  });
+  refresh = $interval(function() {
+    return SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
+      return $scope.metrics = data[0];
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]);
+
+angular.module('flinkApp').directive('livechart', function() {
+  return {
+    link: function(scope, element, attrs) {
+      var getChartOptions, getChartType, getKey1, getKey2, getKey3, getKey4, getYAxisTitle, updateCharts;
+      getChartType = function() {
+        if (attrs.key === "cpuLoad") {
+          return "spline";
+        } else {
+          return "area";
+        }
+      };
+      getYAxisTitle = function() {
+        if (attrs.key === "cpuLoad") {
+          return "CPU Usage(%)";
+        } else {
+          return "Memory(MB)";
+        }
+      };
+      getKey1 = function() {
+        return "memory.total." + attrs.key;
+      };
+      getKey2 = function() {
+        return "memory.heap." + attrs.key;
+      };
+      getKey3 = function() {
+        return "memory.non-heap." + attrs.key;
+      };
+      getKey4 = function() {
+        return "cpuLoad";
+      };
+      getChartOptions = function() {
+        return {
+          title: {
+            text: ' '
+          },
+          chart: {
+            type: getChartType(),
+            zoomType: 'x'
+          },
+          xAxis: {
+            type: 'datetime'
+          },
+          yAxis: {
+            title: {
+              text: getYAxisTitle()
+            },
+            min: attrs.key === "cpuLoad" ? 0 : void 0,
+            max: attrs.key === "cpuLoad" ? 100 : void 0
+          },
+          series: [
+            {
+              name: "Memory: Total",
+              id: getKey1(),
+              data: [],
+              color: "#7cb5ec"
+            }, {
+              name: "Memory: Heap",
+              id: getKey2(),
+              data: [],
+              color: "#434348"
+            }, {
+              name: "Memory: Non-Heap",
+              id: getKey3(),
+              data: [],
+              color: "#90ed7d"
+            }, {
+              name: "CPU Usage",
+              id: getKey4(),
+              data: [],
+              color: "#f7a35c",
+              showInLegend: false
+            }
+          ],
+          legend: {
+            enabled: false
+          },
+          tooltip: {
+            shared: true
+          },
+          exporting: {
+            enabled: false
+          },
+          credits: {
+            enabled: false
+          }
+        };
+      };
+      if (element.highcharts() == null) {
+        element.highcharts(getChartOptions());
+      }
+      scope.$watch(attrs.data, function(value) {
+        return updateCharts(value);
+      });
+      return updateCharts = function(value) {
+        return (function(value) {
+          var chart, divider, heartbeat;
+          heartbeat = value.timeSinceLastHeartbeat;
+          chart = element.highcharts();
+          if (attrs.key === "cpuLoad") {
+            return chart.get(getKey4()).addPoint([heartbeat, +((value.metrics.gauges[getKey4()].value * 100).toFixed(2))], true, false);
+          } else {
+            divider = 1048576;
+            chart.get(getKey1()).addPoint([heartbeat, +((value.metrics.gauges[getKey1()].value / divider).toFixed(2))], true, false);
+            chart.get(getKey2()).addPoint([heartbeat, +((value.metrics.gauges[getKey2()].value / divider).toFixed(2))], true, false);
+            return chart.get(getKey3()).addPoint([heartbeat, +((value.metrics.gauges[getKey3()].value / divider).toFixed(2))], true, false);
+          }
+        })(value);
+      };
+    }
+  };
+});
+
+angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadManagers = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("taskmanagers").success(function(data, status, headers, config) {
+      return deferred.resolve(data['taskmanagers']);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('SingleTaskManagerService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadMetrics = function(taskmanagerid) {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("taskmanagers/" + taskmanagerid).success(function(data, status, headers, config) {
+      return deferred.resolve(data['taskmanagers']);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5
 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQUNILFlBQVksYUFBYSxLQUFLLFNBQUMsUUFBRDtJQUM1QixRQUFRLE9BQU8sYUFBYTtJQUU1QixZQUFZO0lDNUJaLE9EOEJBLFVBQVUsV0FBQTtNQzdCUixPRDhCQSxZQUFZO09BQ1osWUFBWTs7RUFFaEIsV0FBVyxXQUFXO0lBQ3BCLFFBQVE7TUFDTixRQUFROzs7RUFZWixXQUFXLGNBQWMsUUFBUTtJQUNoQyxNQUFNO0lBQ04sS0FBSztJQUNMLE1BQU07S0FDSixNQUFNLFNBQVMscUJBQXFCLFFBQVE7RUFFL0MsV0FBVyxRQUFRO0lBQ2xCLFFBQVEsQ0FBQyxXQUFXLFdBQVcsV0FB
 VyxXQUFXLFdBQVcsV0FBVyxXQUMxRSxXQUFXLFdBQVcsV0FBVztJQUNsQyxPQUFPO01BQ04saUJBQWlCO01BQ2pCLE9BQU87UUFDTixZQUFZOzs7SUFHZCxPQUFPO01BQ04sT0FBTztRQUNOLFVBQVU7UUFDVixZQUFZO1FBQ1osZUFBZTs7O0lBR2pCLFNBQVM7TUFDUixhQUFhO01BQ2IsaUJBQWlCO01BQ2pCLFFBQVE7O0lBRVQsUUFBUTtNQUNQLFdBQVc7UUFDVixZQUFZO1FBQ1osVUFBVTs7O0lBR1osT0FBTztNQUNOLGVBQWU7TUFDZixRQUFRO1FBQ1AsT0FBTztVQUNOLFVBQVU7Ozs7SUFJYixPQUFPO01BQ04sbUJBQW1CO01BQ25CLE9BQU87UUFDTixPQUFPO1VBQ04sZUFBZTs7O01BR2pCLFFBQVE7UUFDUCxPQUFPO1VBQ04sVUFBVTs7OztJQUliLGFBQWE7TUFDWixhQUFhO1FBQ1osV0FBVzs7O0lBSWIsYUFBYTs7RUN4Q2QsT0Q0Q0EsV0FBVyxXQUFXLFdBQVc7SUFLbEMsaUNBQU8sU0FBQyx1QkFBRDtFQy9DTixPRGdEQSxzQkFBc0I7SUFJdkIsZ0RBQU8sU0FBQyxnQkFBZ0Isb0JBQWpCO0VBQ04sZUFBZSxNQUFNLFlBQ25CO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGNBQ0w7SUFBQSxLQUFLO0lBQ
 0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLG1CQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSw0QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0NBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHVCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sOEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFFBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0g7SUFBQSxLQUFLO0
 lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRW5CLE1BQU0sMEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sY0FDSDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7OztLQUVwQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7Ozs7RUMxQmxCLE9ENEJBLG1CQUFtQixVQUFVOztBQzFCL0I7QUNoUEEsUUFBUSxPQUFPLFlBSWQsVUFBVSwyQkFBVyxTQUFDLGFBQUQ7RUNyQnBCLE9Ec0JBO0lBQUEsWUFBWTtJQUNaLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsaUJBQWlCLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJNUQsVUFBVSxvQ0F
 Bb0IsU0FBQyxhQUFEO0VDckI3QixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsc0NBQXNDLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJakYsVUFBVSxpQkFBaUIsV0FBQTtFQ3JCMUIsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLE9BQU87O0lBRVQsVUFBVTs7O0FDbEJaO0FDcEJBLFFBQVEsT0FBTyxZQUVkLE9BQU8sb0RBQTRCLFNBQUMscUJBQUQ7RUFDbEMsSUFBQTtFQUFBLGlDQUFpQyxTQUFDLE9BQU8sUUFBUSxnQkFBaEI7SUFDL0IsSUFBYyxPQUFPLFVBQVMsZUFBZSxVQUFTLE1BQXREO01BQUEsT0FBTzs7SUNoQlAsT0RrQkEsT0FBTyxTQUFTLE9BQU8sUUFBUSxPQUFPLGdCQUFnQjtNQUFFLE1BQU07OztFQUVoRSwrQkFBK0IsWUFBWSxvQkFBb0I7RUNmL0QsT0RpQkE7SUFFRCxPQUFPLGdCQUFnQixXQUFBO0VDakJ0QixPRGtCQSxTQUFDLE1BQUQ7SUFFRSxJQUFHLE1BQUg7TUNsQkUsT0RrQlcsS0FBSyxRQUFRLFNBQVMsS0FBSyxRQUFRLFdBQVU7V0FBMUQ7TUNoQkUsT0RnQmlFOzs7R0FFdEUsT0FBTyxTQUFTLFdBQUE7RUNkZixPRGVBLFNBQUMsT0FBTyxXQUFSO0lBQ0UsSUFBQSxRQUFBO0lBQUEsSUFBZSxNQUFNLFdBQVcsV0FBVyxDQUFJLFNBQVMsUUFBeEQ7TUFBQSxPQUFPOztJQUNQLElBQWtCLE9BQU8sY0FB
 YSxhQUF0QztNQUFBLFlBQVk7O0lBQ1osUUFBUSxDQUFFLFNBQVMsTUFBTSxNQUFNLE1BQU0sTUFBTTtJQUMzQyxTQUFTLEtBQUssTUFBTSxLQUFLLElBQUksU0FBUyxLQUFLLElBQUk7SUNUL0MsT0RVQSxDQUFDLFFBQVEsS0FBSyxJQUFJLE1BQU0sS0FBSyxNQUFNLFVBQVUsUUFBUSxhQUFhLE1BQU0sTUFBTTs7O0FDUGxGO0FDaEJBLFFBQVEsT0FBTyxZQUVkLFFBQVEsOENBQWUsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDdEIsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFVBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9Ec0JBOztBQ3BCRjtBQ09BLFFBQVEsT0FBTyxZQUVkLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQ25CeEMsT0RvQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7SUFDeEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNsQnRCLE9EbUJBLE9BQU8sV0FBVyxZQUFZOztJQUVqQyxXQUFXLGdFQUE0QixTQUFDLFFBQVEsdUJBQVQ7RUFDdEMsc0JBQXNCLFdBQVcsS0FBSyxTQUFDLE1BQUQ7SUFDcEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNqQnRCLE9Ea0JBLE9BQU8sV0FBVyxTQUFTOztFQ2hCN0IsT0RrQkEsT0FBTyxhQUFhLFdBQUE7SUNqQmxCLE9Ea0JBLHNCQUFzQixXQUFXLEtBQ
 UssU0FBQyxNQUFEO01DakJwQyxPRGtCQSxPQUFPLFdBQVcsU0FBUzs7O0lBRWhDLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQUN4Qyx3QkFBd0IsYUFBYSxLQUFLLFNBQUMsTUFBRDtJQUN4QyxJQUFJLE9BQUEsY0FBQSxNQUFKO01BQ0UsT0FBTyxhQUFhOztJQ2Z0QixPRGdCQSxPQUFPLFdBQVcsWUFBWTs7RUNkaEMsT0RnQkEsT0FBTyxhQUFhLFdBQUE7SUNmbEIsT0RnQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7TUNmeEMsT0RnQkEsT0FBTyxXQUFXLFlBQVk7Ozs7QUNacEM7QUNkQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDBEQUEyQixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNsQyxJQUFBO0VBQUEsU0FBUztFQUVULEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxxQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxTQUFTO01DcEJULE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9EcUJBO0lBRUQsUUFBUSx3REFBeUIsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDaEMsSUFBQTtFQUFBLE9BQU87RUFFUCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksa0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsT0FBTztNQ3RCUCxPRHVCQSxTQUFTLFFBQVE7O0lDckJuQixPRHVCQSxTQUFTOztFQ3JCWCxPRHVCQTtJQUVELFFBQVEsME
 RBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHFCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUN4QlQsT0R5QkEsU0FBUyxRQUFROztJQ3ZCbkIsT0R5QkEsU0FBUzs7RUN2QlgsT0R5QkE7O0FDdkJGO0FDdEJBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyxxSEFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQVksYUFBYSxXQUFyRTtFQUNqQyxJQUFBO0VBQUEsUUFBUSxJQUFJO0VBRVosT0FBTyxRQUFRLGFBQWE7RUFDNUIsT0FBTyxNQUFNO0VBQ2I
 sT0FBTyxPQUFPO0VBQ2QsT0FBTyxXQUFXO0VBRWxCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUFDM0MsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPLEtBQUs7SUMxQm5CLE9EMkJBLE9BQU8sV0FBVyxLQUFLOztFQUV6QixZQUFZLFVBQVUsV0FBQTtJQzFCcEIsT0QyQkEsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtNQUMzQyxPQUFPLE1BQU07TUMxQmIsT0Q0QkEsT0FBTyxXQUFXOztLQUVwQixZQUFZO0VDM0JkLE9ENkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUFDckIsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPO0lBQ2QsT0FBTyxXQUFXO0lDNUJsQixPRDhCQSxVQUFVLE9BQU87O0lBS3BCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDL0IsUUFBUSxJQUFJO0VBRVosT0FBTyxTQUFTO0VBQ2hCLE9BQU8sZUFBZTtFQUN0QixPQUFPLFlBQVksWUFBWTtFQUUvQixPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01BQ2hCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUFDbEIsT0FBTyxlQUFlO01DbEN0QixPRG9DQSxPQUFPLFdBQVc7V0FOcEI7TUFTRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxlQUFlO01BQ3RCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUNwQ2xCLE9EcUNBLE9BQU8sZUFBZTs7O0VBRTFCLE9BQU8saUJBQWlCLFdBQUE7SUFDdEIsT0FBTyxTQUFTO0lBQ2hCLE9BQU8s
 ZUFBZTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxXQUFXO0lDbkNsQixPRG9DQSxPQUFPLGVBQWU7O0VDbEN4QixPRG9DQSxPQUFPLGFBQWEsV0FBQTtJQ25DbEIsT0RvQ0EsT0FBTyxlQUFlLENBQUMsT0FBTzs7SUFJakMsV0FBVyx1REFBNkIsU0FBQyxRQUFRLGFBQVQ7RUFDdkMsUUFBUSxJQUFJO0VBRVosSUFBRyxPQUFPLFdBQVksQ0FBQyxPQUFPLFVBQVUsQ0FBQyxPQUFPLE9BQU8sS0FBdkQ7SUFDRSxZQUFZLFlBQVksT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01DdEMxQyxPRHVDQSxPQUFPLFdBQVc7OztFQ3BDdEIsT0RzQ0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQUNaLElBQUcsT0FBTyxRQUFWO01DckNFLE9Ec0NBLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUNyQzFDLE9Ec0NBLE9BQU8sV0FBVzs7OztJQUl6QixXQUFXLDJEQUFpQyxTQUFDLFFBQVEsYUFBVDtFQUMzQyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxlQUF2RDtJQUNFLFlBQVksZ0JBQWdCLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtNQUM5QyxPQUFPLGVBQWUsS0FBSztNQ3RDM0IsT0R1Q0EsT0FBTyxzQkFBc0IsS0FBSzs7O0VDcEN0QyxPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRL
 EtBQUssU0FBQyxNQUFEO1FBQzlDLE9BQU8sZUFBZSxLQUFLO1FDckMzQixPRHNDQSxPQUFPLHNCQUFzQixLQUFLOzs7O0lBSXpDLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDekMsUUFBUSxJQUFJO0VBRVosWUFBWSxVQUFVLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ3RDaEQsT0R1Q0EsT0FBTyxTQUFTOztFQ3JDbEIsT0R1Q0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQ3RDWixPRHVDQSxZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO01DdENoRCxPRHVDQSxPQUFPLFNBQVM7OztJQUlyQixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDdkNyQyxPRHdDQSxZQUFZLGlCQUFpQixLQUFLLFNBQUMsTUFBRDtJQ3ZDaEMsT0R3Q0EsT0FBTyxhQUFhOztJQUl2QixXQUFXLHFEQUEyQixTQUFDLFFBQVEsYUFBVDtFQUNyQyxRQUFRLElBQUk7RUN6Q1osT0QyQ0EsT0FBTyxhQUFhLFNBQUMsUUFBRDtJQUNsQixJQUFHLFdBQVUsT0FBTyxRQUFwQjtNQUNFLE9BQU8sU0FBUztNQzFDaEIsT0Q0Q0EsWUFBWSxRQUFRLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUMzQy9CLE9ENENBLE9BQU8sT0FBTzs7V0FKbEI7TUFPRSxPQUFPLFNBQVM7TUMzQ2hCLE9ENENBLE9BQU8sT0FBTzs7OztBQ3hDcEI7QUNuSEEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TU
 FBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBO01BQUEsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUztNQUVyQyxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsT0FBQSxLQUFBO1FBQUEsR0FBRyxPQUFPLE9BQU8sVUFBVSxLQUFLO1FBRWhDLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsU0FBUyxHQUFWO1VBQzdCLElBQUE7VUFBQSxRQUFRO1lBQ047Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNO2VBRVI7Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUFJVixJQUFHLFFBQVEsV0FBVyxjQUFjLEdBQXBDO1lBQ0UsTUFBTSxLQUFLO2NBQ1QsT0FBTztjQUNQLE9BQU87Y0FDUCxhQUFhO2NBQ2IsZUFBZSxRQUFRLFdBQVc7Y0FDbEMsYUFBYSxRQUFRLFdBQVc7Y0FDaEMsTUFBTTs7O1VDdEJSLE9EeUJGLFNBQVMsS0FBSztZQUNaLE9BQU8sTUFBSSxRQUFRLFVBQVEsT0FBSSxRQUFRO1lBQ3ZDLE9BQU87OztRQUdYLFFBQVEsR0FBRyxXQUFXLFFBQ3JCLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLFVBQVU7V0FFWCxPQUFPLFVBQ1AsWUFBWSxTQUFDLE9BQUQ7VUM1QlQ
 sT0Q2QkY7V0FFRCxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTtXQUM5QyxXQUFXLElBQ1g7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQ2hDckIsT0RpQ0E7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLFVBQVU7TUFDVixPQUFPOztJQUVULE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGlCQUFpQixTQUFDLE9BQUQ7UUNqQ2IsT0RrQ0YsTUFBTSxRQUFRLFFBQVE7O01BRXhCLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxNQUFNLFNBQUMsUUFBRDtVQUNwQixJQUFHLE9BQU8sZ0JBQWdCLENBQUMsR0FBM0I7WUFDRSxJQUFHLE9BQU8sU0FBUSxhQUFsQjtjQ2xDSSxPRG1DRixTQUFTLEtBQ1A7Z0JBQUEsT0FBTztrQkFDTDtvQkFBQSxPQUFPLGVBQWUsT0FBTztvQkFDN0IsT0FBTztvQkFDUCxhQUFhO29CQUNiLGVBQWUsT0FBTztvQkFDdEIsYUFBYSxPQUFPO29CQUNwQixNQUFNLE9BQU87Ozs7bUJBUm5CO2NDckJJLE9EZ0NGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29C
 QUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTztvQkFDYixNQUFNLE9BQU87Ozs7Ozs7UUFHdkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUFDbEMsSUFBRyxFQUFFLE1BQUw7WUMxQkksT0QyQkYsT0FBTyxHQUFHLDhCQUE4QjtjQUFFLE9BQU8sTUFBTTtjQUFPLFVBQVUsRUFBRTs7O1dBRzdFLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBR3ZCLFVBQVU7V0FFWCxPQUFPLFFBQ1AsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7V0FDNUMsV0FBVyxJQUNYLGlCQUNBO1FDMUJDLE9ENEJGLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUs7O01BRVIsTUFBTSxPQUFPLE1BQU0sVUFBVSxTQUFDLE1BQUQ7UUFDM0IsSUFBcUIsTUFBckI7VUM3QkksT0Q2QkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUM3QnBCLE9EOEJBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNO01BQ04sU0FBUzs7SUFFWCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsR0FBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLElBQUk7TUFDSixXQ
 UFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFLckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDMUN2QixPRDZDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM1Q3ZCLE9EK0NGLFdBQVcsS0FBSyxhQU
 FhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3RDZCxPRHVDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDeENyQixPRHlDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDeENJLE9EeUNGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUN6Q0QsT0QwQ0Y7ZUFERztVQ3ZDRCxPRDJDQTs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFFaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSx1QkFBdUIsUUFBUSxhQUFhLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHbEgsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLFdBQVc7ZUFEbkU7VUFHRSxjQUFjLDJCQUEyQixHQUFHLFdBQVc7O1FBQ3pELElBQUcsR0FBRyxnQkFBZSxJQUF
 yQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUF3RixHQUFHLGFBQVksV0FBdkc7WUFBQSxjQUFjLG9CQUFvQixjQUFjLEdBQUcscUJBQXFCOzs7UUFHMUUsY0FBYztRQzNDWixPRDRDRjs7TUFHRiw4QkFBOEIsU0FBQyxJQUFJLE1BQU0sTUFBWDtRQUM1QixJQUFBLFlBQUE7UUFBQSxRQUFRLFNBQVM7UUFFakIsYUFBYSxpQkFBaUIsUUFBUSxhQUFhLE9BQU8sYUFBYSxPQUFPO1FDNUM1RSxPRDZDRjs7TUFHRixnQkFBZ0IsU0FBQyxHQUFEO1FBRWQsSUFBQTtRQUFBLElBQUcsRUFBRSxPQUFPLE9BQU0sS0FBbEI7VUFDRSxJQUFJLEVBQUUsUUFBUSxLQUFLO1VBQ25CLElBQUksRUFBRSxRQUFRLEtBQUs7O1FBQ3JCLE1BQU07UUFDTixPQUFNLEVBQUUsU0FBUyxJQUFqQjtVQUNFLE1BQU0sTUFBTSxFQUFFLFVBQVUsR0FBRyxNQUFNO1VBQ2pDLElBQUksRUFBRSxVQUFVLElBQUksRUFBRTs7UUFDeEIsTUFBTSxNQUFNO1FDM0NWLE9ENENGOztNQUVGLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxVQUFrQixNQUFNLE1BQXRDO1FDM0NU
 LElBQUksWUFBWSxNQUFNO1VEMkNDLFdBQVc7O1FBRXBDLElBQUcsR0FBRyxPQUFNLEtBQUssa0JBQWpCO1VDekNJLE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLG1CQUFtQixNQUFNO1lBQ3BELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyx1QkFBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksdUJBQXVCLE1BQU07WUFDeEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLFNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLFdBQVcsTUFBTTtZQUM1QyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGdCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBT
 yxnQkFBZ0IsSUFBSSxpQkFBaUIsTUFBTTtZQUNsRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBSnRCO1VDbkNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLElBQUksTUFBTTtZQUNyQyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7Ozs7TUFFN0IsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTSxjQUFuQztRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3RDSSxPRHVDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFFdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxhQUFhLFFBQVEsWUFBWSxNQUFNLENBQUMsSUFBL0Q7WUFDRSxhQUFhLEtBQUssWUFBWTtZQUM5QixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ3RDaEMsT0R3Q0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsY0FBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBQ2hCLGVBQWU7UUFFZixJQUFHLE
 tBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUN2Q0ksS0FBSyxVQUFVO1VEd0NqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLFVBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDMUNJLE9BQU8sSUFBSTtjRDJDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTTs7OztRQ3RDakQsT0R3Q0Y7O01BR0YsZ0JBQWdCLFNBQUMsTUFBTSxRQUFQO1FBQ2QsSUFBQSxJQUFBLEdBQUE7UUFBQSxLQUFBLEtBQUEsS0FBQSxPQUFBO1VBQ0UsS0FBSyx
 LQUFLLE1BQU07VUFDaEIsSUFBYyxHQUFHLE9BQU0sUUFBdkI7WUFBQSxPQUFPOztVQUdQLElBQUcsR0FBQSxpQkFBQSxNQUFIO1lBQ0UsS0FBQSxLQUFBLEdBQUEsZUFBQTtjQUNFLElBQStCLEdBQUcsY0FBYyxHQUFHLE9BQU0sUUFBekQ7Z0JBQUEsT0FBTyxHQUFHLGNBQWM7Ozs7OztNQUVoQyxZQUFZLFNBQUMsTUFBRDtRQUNWLElBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQ2pDSSxLQUFLLFVBQVU7VURrQ2pCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdB
 QUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ25DTixPRG9DRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUFFckYsU0FBUztRQ25DUCxPRHFDRixXQUFXLFVBQVUsU0FBUyxHQUFHLFNBQVMsU0FBQyxHQUFEO1VDcEN0QyxPRHFDRixNQUFNLFFBQVE7WUFBRSxRQUFROzs7O01BRTVCLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDakNJLE9EaUNKLFVBQVU7Ozs7OztBQzNCaEI7QUMxYUEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBRWQsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3JCaEIsT0RzQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DckI1QixPRHNCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDcEJsQixPRHFCQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNuQjdCLE9Eb0JBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ25CWCxPRG9CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc
 0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDNUJILE9ENEJtQjtNQUR2QixLQUVPO1FDM0JILE9EMkJpQjtNQUZyQixLQUdPO1FDMUJILE9EMEJvQjtNQUh4QixLQUlPO1FDekJILE9EeUJvQjtNQUp4QixLQUtPO1FDeEJILE9Ed0JrQjtNQUx0QixLQU1PO1FDdkJILE9EdUJvQjtNQU54QixLQU9PO1FDdEJILE9Ec0JrQjtNQVB0QixLQVFPO1FDckJILE9EcUJnQjtNQVJwQjtRQ1hJLE9Eb0JHOzs7RUFFVCxLQUFDLGNBQWMsU0FBQyxNQUFEO0lDbEJiLE9EbUJBLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxRQUFQO01BQ3BCLElBQUEsRUFBTyxLQUFLLGNBQWMsQ0FBQyxJQUEzQjtRQ2xCRSxPRG1CQSxLQUFLLGNBQWMsS0FBSyxnQkFBZ0IsS0FBSzs7OztFQUVuRCxLQUFDLGtCQUFrQixTQUFDLE1BQUQ7SUFDakIsUUFBUSxRQUFRLEtBQUssVUFBVSxTQUFDLFFBQVEsR0FBVDtNQ2hCN0IsT0RpQkEsT0FBTyxPQUFPOztJQ2ZoQixPRGlCQSxLQUFLLFNBQVMsUUFBUTtNQUNwQixNQUFNO01BQ04sY0FBYyxLQUFLLFdBQVc7TUFDOUIsWUFBWSxLQUFLLFdBQVcsYUFBYTtNQUN6QyxNQUFNOzs7RUFHVixLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZUFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1
 lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxVQUFVLE9BQ25CLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFVBQVUsUUFBUSxXQUMzQixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE
 7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQyxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQzlELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RF
 RixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsVUFDbkQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDOUQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDOUQsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ES
 nlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUNwQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFlBQ1
 QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLDJGQUE2QixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDdkMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDakJkLE9EbUJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNsQnJCLE9EbUJBLFVBQVUsT0FBTzs7SUFFcEIsV0FBVyxrSEFBK0IsU0FBQyxRQUFRLGNBQWMsMEJBQTBCLFdBQVcsYUFBNUQ7RUFDekMsSUFBQTtFQUFBLE9BQU8sVUFBVTtFQUNqQix5QkFBeUIsWUFBWSxhQUFhLGVBQWUsS0FBSyxTQUFDLE1BQUQ7SUNqQnBFLE9Ea0JFLE9BQU8sVUFBVSxLQUFLOztFQUV4QixVQUFVLFVBQVUsV0FBQTtJQ2pCcEIsT0RrQkUseUJBQXlCLFlBQVksYUFBYSxlQUFlLEtBQUssU0FBQyxNQUFEO01DakJ0RSxPRGtCRSxPQUFPLFVBQVUsS0FBSzs7S0FDeEIsWUFBWTtFQ2hCaEIsT0RrQkUsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2pCdkIsT0RrQkUsVUFBVSxPQUFPOzs7QUNmdkI7QUNWQSxRQUFRLE9BQU8sWUFFZCxVQUFVLGFBQWEsV0FBQTtFQ25CdEI
 sT0RvQkE7SUFDRSxNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01BQ0osSUFBQSxpQkFBQSxjQUFBLFNBQUEsU0FBQSxTQUFBLFNBQUEsZUFBQTtNQUFBLGVBQWUsV0FBQTtRQUNiLElBQUcsTUFBTSxRQUFPLFdBQWhCO1VDbEJFLE9EbUJBO2VBREY7VUNoQkUsT0RtQkE7OztNQUVKLGdCQUFnQixXQUFBO1FBQ2QsSUFBRyxNQUFNLFFBQU8sV0FBaEI7VUNqQkUsT0RrQkE7ZUFERjtVQ2ZFLE9Ea0JBOzs7TUFFSixVQUFVLFdBQUE7UUNoQlIsT0RpQkEsa0JBQWtCLE1BQU07O01BQzFCLFVBQVUsV0FBQTtRQ2ZSLE9EZ0JBLGlCQUFpQixNQUFNOztNQUN6QixVQUFVLFdBQUE7UUNkUixPRGVBLHFCQUFxQixNQUFNOztNQUM3QixVQUFVLFdBQUE7UUNiUixPRGNBOztNQUVGLGtCQUFrQixXQUFBO1FDYmhCLE9EYXNCO1VBQ3RCLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFBQyxNQUFNO1lBQWdCLFVBQVU7O1VBQ3hDLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFDTCxPQUFPO2NBQUMsTUFBTTs7WUFDZCxLQUFVLE1BQU0sUUFBTyxZQUFsQixJQUFBLEtBQUE7WUFDTCxLQUFZLE1BQU0sUUFBTyxZQUFwQixNQUFBLEtBQUE7O1VBRVAsUUFBUTtZQUNOO2NBQUMsTUFBTTtjQUFpQixJQUFJO2NBQVcsTUFBTTtjQUFJLE9BQU87ZUFDeEQ7Y0FBQyxNQUFNO2NBQWdCLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztlQUN2RDtjQUFDLE1BQU07Y0FBb0IsSUFBSTtjQUFXLE1BQU07Y0FBSSxPQUFPO2VBQzNEO2NBQUMsTUFB
 TTtjQUFhLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztjQUFXLGNBQWM7OztVQUUvRSxRQUFRO1lBQUMsU0FBUzs7VUFDbEIsU0FBUztZQUFDLFFBQVE7O1VBQ2xCLFdBQVc7WUFBQyxTQUFTOztVQUNyQixTQUFTO1lBQUMsU0FBUzs7OztNQUdyQixJQUFJLFFBQUEsZ0JBQUEsTUFBSjtRQUNFLFFBQVEsV0FBVzs7TUFFckIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLE9BQUQ7UUN1QnZCLE9EdEJBLGFBQWE7O01Dd0JmLE9EckJBLGVBQWUsU0FBQyxPQUFEO1FDc0JiLE9EckJFLENBQUEsU0FBQyxPQUFEO1VBQ0EsSUFBQSxPQUFBLFNBQUE7VUFBQSxZQUFZLE1BQU07VUFDbEIsUUFBUSxRQUFRO1VBQ2hCLElBQUcsTUFBTSxRQUFPLFdBQWhCO1lDdUJFLE9EdEJBLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLEtBQUssUUFBUSxNQUNsRSxNQUFNO2lCQUhYO1lBS0UsVUFBVTtZQUNWLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLFNBQVMsUUFBUSxNQUN0RSxNQUFNO1lBQ1QsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQUFRLE1BQ3RFLE1BQU07WUNpQlQsT0RoQkEsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQ
 UFRLE1BQ3RFLE1BQU07O1dBakJWOzs7OztBQ3NDWDtBQzdGQSxRQUFRLE9BQU8sWUFFZCxRQUFRLHNEQUF1QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUM5QixLQUFDLGVBQWUsV0FBQTtJQUNkLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZ0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUSxLQUFLOztJQ25CeEIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7SUFFRCxRQUFRLDJEQUE0QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNuQyxLQUFDLGNBQWMsU0FBQyxlQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxrQkFBa0IsZUFDM0IsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DdEJQLE9EdUJBLFNBQVMsUUFBUSxLQUFLOztJQ3JCeEIsT0R1QkEsU0FBUzs7RUNyQlgsT0R1QkE7O0FDckJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbG
 VcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5ydW4gKCRyb290U2NvcGUpIC0+XHJcbiAgJHJvb3RTY29wZS5zaWR
 lYmFyVmlzaWJsZSA9IGZhbHNlXHJcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XHJcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcclxuICAgICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4udmFsdWUgJ2ZsaW5rQ29uZmlnJywge1xyXG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxyXG59XHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4ucnVuIChKb2JzU2VydmljZSwgTWFpblNlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIC0+XHJcbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cclxuICAgIGFuZ3VsYXIuZXh0ZW5kIGZsaW5rQ29uZmlnLCBjb25maWdcclxuXHJcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXHJcblxyXG4gICAgJGludGVydmFsIC0+XHJcbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcclxuICAgICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gIEhpZ2hjaGFydHMuc2V0T3B0aW9ucyh7XHJcbiAgICBnbG9iYWw6IHtcclxuICAgICAgdXNlVVRDOiBmYWxzZVxyXG4gICAgfVxyXG4gIH0pXHJcblxyXG4gICNcclxuICAjIEdyaWQtbGlnaHQgdGhlbWUgZm9yIEhpZ2hj
 aGFydHMgSlNcclxuICAjIEBhdXRob3IgVG9yc3RlaW4gSG9uc2lcclxuICAjXHJcbiAgIyBUYWtlbiBmcm9tIGh0dHBzOi8vZ2l0aHViLmNvbS9oaWdoc2xpZGUtc29mdHdhcmUvaGlnaGNoYXJ0cy5jb21cclxuICAjXHJcblxyXG5cclxuICBIaWdoY2hhcnRzLmNyZWF0ZUVsZW1lbnQoJ2xpbmsnLCB7XHJcbiAgXHRocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxyXG4gIFx0cmVsOiAnc3R5bGVzaGVldCcsXHJcbiAgXHR0eXBlOiAndGV4dC9jc3MnXHJcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XHJcblxyXG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XHJcbiAgXHRjb2xvcnM6IFtcIiM3Y2I1ZWNcIiwgXCIjZjdhMzVjXCIsIFwiIzkwZWU3ZVwiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCIsIFwiI2ZmMDA2NlwiLCBcIiNlZWFhZWVcIixcclxuICBcdFx0XCIjNTVCRjNCXCIsIFwiI0RGNTM1M1wiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCJdLFxyXG4gIFx0Y2hhcnQ6IHtcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiBudWxsLFxyXG4gIFx0XHRzdHlsZToge1xyXG4gIFx0XHRcdGZvbnRGYW1pbHk6IFwiRG9zaXMsIHNhbnMtc2VyaWZcIlxyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dGl0bGU6IHtcclxuICBcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRmb250U2l6ZTogJzE2cHgnLFxyXG4gIFx0XHRcd
 GZvbnRXZWlnaHQ6ICdib2xkJyxcclxuICBcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dG9vbHRpcDoge1xyXG4gIFx0XHRib3JkZXJXaWR0aDogMCxcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiAncmdiYSgyMTksMjE5LDIxNiwwLjgpJyxcclxuICBcdFx0c2hhZG93OiBmYWxzZVxyXG4gIFx0fSxcclxuICBcdGxlZ2VuZDoge1xyXG4gIFx0XHRpdGVtU3R5bGU6IHtcclxuICBcdFx0XHRmb250V2VpZ2h0OiAnYm9sZCcsXHJcbiAgXHRcdFx0Zm9udFNpemU6ICcxM3B4J1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0eEF4aXM6IHtcclxuICBcdFx0Z3JpZExpbmVXaWR0aDogMSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHR5QXhpczoge1xyXG4gIFx0XHRtaW5vclRpY2tJbnRlcnZhbDogJ2F1dG8nLFxyXG4gIFx0XHR0aXRsZToge1xyXG4gIFx0XHRcdHN0eWxlOiB7XHJcbiAgXHRcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHRcdH1cclxuICBcdFx0fSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHRwbG90T3B0aW9ucz
 oge1xyXG4gIFx0XHRjYW5kbGVzdGljazoge1xyXG4gIFx0XHRcdGxpbmVDb2xvcjogJyM0MDQwNDgnXHJcbiAgXHRcdH1cclxuICBcdH0sXHJcblxyXG4gIFx0YmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xyXG5cclxuICB9O1xyXG5cclxuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoSGlnaGNoYXJ0cy50aGVtZSk7XHJcblxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxyXG4gICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cclxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXHJcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxyXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgIG1haW46XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5
 nLWpvYnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcclxuICBcclxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxyXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgbWFpbjpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxyXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcclxuICAgIHVybDogXCJcIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIixcclxuICAgIHVybDogXCJc
 IlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0Lm92ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLFxyXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcclxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInIFxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXHJcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxyXG4gICAgdXJsOiBcIi97dmVydGV4SWR9XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICB2ZXJ0ZXg6XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4L
 mh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLFxyXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IuZXhjZXB0aW9uc1wiLFxyXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIixcclxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOl
 xyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiYWxsLW1hbmFnZXJcIixcclxuICAgIHVybDogXCIvdGFza21hbmFnZXJzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL2luZGV4Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL3Rhc2ttYW5hZ2VyL3t0YXNrbWFuYWdlcmlkfVwiXHJcbiAgICAgIHZpZXdzOlxyXG4gICAgICAgIG1haW46XHJcbiAgICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5odG1sXC

<TRUNCATED>

[45/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot
deleted file mode 100644
index 33b2bb8..0000000
Binary files a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.eot and /dev/null differ


[29/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
index 9b33c6a..c586db3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
@@ -28,8 +28,8 @@ import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.state.StateBackend;
-import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 29bf938..e953696 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -26,7 +26,7 @@ import org.apache.flink.api.common.io.{FileInputFormat, InputFormat}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
 import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.state.StateBackend
+import org.apache.flink.runtime.state.StateBackend
 import org.apache.flink.streaming.api.{TimeCharacteristic, CheckpointingMode}
 import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaEnv}
 import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index 989db14..3b5295f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -23,7 +23,7 @@ import java.io.File;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.test.testdata.KMeansData;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
index 2cdf83c..f15644e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
@@ -542,7 +543,7 @@ public class ChaosMonkeyITCase {
 
 		LOG.info("Checking file system backend state...");
 
-		File fsCheckpoints = new File(config.getString(ConfigConstants.STATE_BACKEND_FS_DIR, ""));
+		File fsCheckpoints = new File(config.getString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, ""));
 
 		LOG.info("Checking " + fsCheckpoints);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
index 54ddf7e..e590067 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java
@@ -44,6 +44,7 @@ import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
@@ -68,7 +69,7 @@ import static org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-public class JobManagerCheckpointRecoveryITCase {
+public class JobManagerCheckpointRecoveryITCase extends TestLogger {
 
 	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
 
@@ -144,7 +145,7 @@ public class JobManagerCheckpointRecoveryITCase {
 		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
 
 		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(ZooKeeper
-				.getConnectString(), FileStateBackendBasePath.getPath());
+				.getConnectString(), FileStateBackendBasePath.getAbsoluteFile().toURI().toString());
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Parallelism);
 
 		ActorSystem testSystem = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
index aa634f0..63aa967 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
@@ -33,7 +33,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 
 import org.junit.Assert;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
index 5c7a932..5840a98 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -95,7 +95,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
-		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath());
+		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getAbsoluteFile().toURI().toString());
 
 		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
 
@@ -144,7 +144,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
 		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
-		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath());
+		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getAbsoluteFile().toURI().toString());
 
 		// we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make
 		// sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-tests/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/log4j-test.properties b/flink-tests/src/test/resources/log4j-test.properties
index 3ba6d1d..62807b3 100644
--- a/flink-tests/src/test/resources/log4j-test.properties
+++ b/flink-tests/src/test/resources/log4j-test.properties
@@ -19,7 +19,6 @@
 # Set root logger level to OFF to not flood build logs
 # set manually to INFO for debugging purposes
 log4j.rootLogger=INFO, testlogger
-log4j.logger.org.apache.flink.runtime.client.JobClientActor=DEBUG
 
 # A1 is set to be a ConsoleAppender.
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index a05621a..ffb43f8 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
@@ -114,7 +115,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 		flinkYarnClient.setDynamicPropertiesEncoded("recovery.mode=zookeeper@@ha.zookeeper.quorum=" +
 			zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts +
 			"@@" + ConfigConstants.STATE_BACKEND + "=FILESYSTEM" +
-			"@@" + ConfigConstants.STATE_BACKEND_FS_DIR + "=" + fsStateHandlePath + "/checkpoints" +
+			"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
 			"@@" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "=" + fsStateHandlePath + "/recovery");
 		flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterBase.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterBase.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterBase.scala
index 84198ec..f3892dd 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterBase.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterBase.scala
@@ -123,7 +123,7 @@ abstract class ApplicationMasterBase {
         config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0); // set port to 0.
       }
 
-      val (actorSystem, jmActor, archivActor, webMonitor) =
+      val (actorSystem, jmActor, archiveActor, webMonitor) =
         JobManager.startActorSystemAndJobManagerActors(
           config,
           JobManagerMode.CLUSTER,

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 47c47c5..efe2101 100644
--- a/pom.xml
+++ b/pom.xml
@@ -735,7 +735,7 @@ under the License.
 						<exclude>flink-runtime-web/web-dashboard/assets/fonts/fontawesome*</exclude>
 
                         <!-- generated contents -->
-						<exclude>flink-runtime-web/src/main/resources/web/**</exclude>
+						<exclude>flink-runtime-web/web-dashboard/web/**</exclude>
 
 						<!-- downloaded and generated web libraries. -->
 						<exclude>flink-runtime-web/web-dashboard/node_modules/**</exclude>


[24/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/css/vendor.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/css/vendor.css b/flink-runtime-web/src/main/resources/web/css/vendor.css
new file mode 100644
index 0000000..672e07f
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/css/vendor.css
@@ -0,0 +1,9183 @@
+/*!
+ *  Font Awesome 4.3.0 by @davegandy - http://fontawesome.io - @fontawesome
+ *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
+ */
+/* FONT PATH
+ * -------------------------- */
+@font-face {
+  font-family: 'FontAwesome';
+  src: url('../fonts/fontawesome-webfont.eot?v=4.3.0');
+  src: url('../fonts/fontawesome-webfont.eot?#iefix&v=4.3.0') format('embedded-opentype'), url('../fonts/fontawesome-webfont.woff2?v=4.3.0') format('woff2'), url('../fonts/fontawesome-webfont.woff?v=4.3.0') format('woff'), url('../fonts/fontawesome-webfont.ttf?v=4.3.0') format('truetype'), url('../fonts/fontawesome-webfont.svg?v=4.3.0#fontawesomeregular') format('svg');
+  font-weight: normal;
+  font-style: normal;
+}
+.fa {
+  display: inline-block;
+  font: normal normal normal 14px/1 FontAwesome;
+  font-size: inherit;
+  text-rendering: auto;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+  transform: translate(0, 0);
+}
+/* makes the font 33% larger relative to the icon container */
+.fa-lg {
+  font-size: 1.33333333em;
+  line-height: 0.75em;
+  vertical-align: -15%;
+}
+.fa-2x {
+  font-size: 2em;
+}
+.fa-3x {
+  font-size: 3em;
+}
+.fa-4x {
+  font-size: 4em;
+}
+.fa-5x {
+  font-size: 5em;
+}
+.fa-fw {
+  width: 1.28571429em;
+  text-align: center;
+}
+.fa-ul {
+  padding-left: 0;
+  margin-left: 2.14285714em;
+  list-style-type: none;
+}
+.fa-ul > li {
+  position: relative;
+}
+.fa-li {
+  position: absolute;
+  left: -2.14285714em;
+  width: 2.14285714em;
+  top: 0.14285714em;
+  text-align: center;
+}
+.fa-li.fa-lg {
+  left: -1.85714286em;
+}
+.fa-border {
+  padding: .2em .25em .15em;
+  border: solid 0.08em #eeeeee;
+  border-radius: .1em;
+}
+.pull-right {
+  float: right;
+}
+.pull-left {
+  float: left;
+}
+.fa.pull-left {
+  margin-right: .3em;
+}
+.fa.pull-right {
+  margin-left: .3em;
+}
+.fa-spin {
+  -webkit-animation: fa-spin 2s infinite linear;
+  animation: fa-spin 2s infinite linear;
+}
+.fa-pulse {
+  -webkit-animation: fa-spin 1s infinite steps(8);
+  animation: fa-spin 1s infinite steps(8);
+}
+@-webkit-keyframes fa-spin {
+  0% {
+    -webkit-transform: rotate(0deg);
+    transform: rotate(0deg);
+  }
+  100% {
+    -webkit-transform: rotate(359deg);
+    transform: rotate(359deg);
+  }
+}
+@keyframes fa-spin {
+  0% {
+    -webkit-transform: rotate(0deg);
+    transform: rotate(0deg);
+  }
+  100% {
+    -webkit-transform: rotate(359deg);
+    transform: rotate(359deg);
+  }
+}
+.fa-rotate-90 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=1);
+  -webkit-transform: rotate(90deg);
+  -ms-transform: rotate(90deg);
+  transform: rotate(90deg);
+}
+.fa-rotate-180 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2);
+  -webkit-transform: rotate(180deg);
+  -ms-transform: rotate(180deg);
+  transform: rotate(180deg);
+}
+.fa-rotate-270 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=3);
+  -webkit-transform: rotate(270deg);
+  -ms-transform: rotate(270deg);
+  transform: rotate(270deg);
+}
+.fa-flip-horizontal {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);
+  -webkit-transform: scale(-1, 1);
+  -ms-transform: scale(-1, 1);
+  transform: scale(-1, 1);
+}
+.fa-flip-vertical {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);
+  -webkit-transform: scale(1, -1);
+  -ms-transform: scale(1, -1);
+  transform: scale(1, -1);
+}
+:root .fa-rotate-90,
+:root .fa-rotate-180,
+:root .fa-rotate-270,
+:root .fa-flip-horizontal,
+:root .fa-flip-vertical {
+  filter: none;
+}
+.fa-stack {
+  position: relative;
+  display: inline-block;
+  width: 2em;
+  height: 2em;
+  line-height: 2em;
+  vertical-align: middle;
+}
+.fa-stack-1x,
+.fa-stack-2x {
+  position: absolute;
+  left: 0;
+  width: 100%;
+  text-align: center;
+}
+.fa-stack-1x {
+  line-height: inherit;
+}
+.fa-stack-2x {
+  font-size: 2em;
+}
+.fa-inverse {
+  color: #ffffff;
+}
+/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen
+   readers do not read off random characters that represent icons */
+.fa-glass:before {
+  content: "\f000";
+}
+.fa-music:before {
+  content: "\f001";
+}
+.fa-search:before {
+  content: "\f002";
+}
+.fa-envelope-o:before {
+  content: "\f003";
+}
+.fa-heart:before {
+  content: "\f004";
+}
+.fa-star:before {
+  content: "\f005";
+}
+.fa-star-o:before {
+  content: "\f006";
+}
+.fa-user:before {
+  content: "\f007";
+}
+.fa-film:before {
+  content: "\f008";
+}
+.fa-th-large:before {
+  content: "\f009";
+}
+.fa-th:before {
+  content: "\f00a";
+}
+.fa-th-list:before {
+  content: "\f00b";
+}
+.fa-check:before {
+  content: "\f00c";
+}
+.fa-remove:before,
+.fa-close:before,
+.fa-times:before {
+  content: "\f00d";
+}
+.fa-search-plus:before {
+  content: "\f00e";
+}
+.fa-search-minus:before {
+  content: "\f010";
+}
+.fa-power-off:before {
+  content: "\f011";
+}
+.fa-signal:before {
+  content: "\f012";
+}
+.fa-gear:before,
+.fa-cog:before {
+  content: "\f013";
+}
+.fa-trash-o:before {
+  content: "\f014";
+}
+.fa-home:before {
+  content: "\f015";
+}
+.fa-file-o:before {
+  content: "\f016";
+}
+.fa-clock-o:before {
+  content: "\f017";
+}
+.fa-road:before {
+  content: "\f018";
+}
+.fa-download:before {
+  content: "\f019";
+}
+.fa-arrow-circle-o-down:before {
+  content: "\f01a";
+}
+.fa-arrow-circle-o-up:before {
+  content: "\f01b";
+}
+.fa-inbox:before {
+  content: "\f01c";
+}
+.fa-play-circle-o:before {
+  content: "\f01d";
+}
+.fa-rotate-right:before,
+.fa-repeat:before {
+  content: "\f01e";
+}
+.fa-refresh:before {
+  content: "\f021";
+}
+.fa-list-alt:before {
+  content: "\f022";
+}
+.fa-lock:before {
+  content: "\f023";
+}
+.fa-flag:before {
+  content: "\f024";
+}
+.fa-headphones:before {
+  content: "\f025";
+}
+.fa-volume-off:before {
+  content: "\f026";
+}
+.fa-volume-down:before {
+  content: "\f027";
+}
+.fa-volume-up:before {
+  content: "\f028";
+}
+.fa-qrcode:before {
+  content: "\f029";
+}
+.fa-barcode:before {
+  content: "\f02a";
+}
+.fa-tag:before {
+  content: "\f02b";
+}
+.fa-tags:before {
+  content: "\f02c";
+}
+.fa-book:before {
+  content: "\f02d";
+}
+.fa-bookmark:before {
+  content: "\f02e";
+}
+.fa-print:before {
+  content: "\f02f";
+}
+.fa-camera:before {
+  content: "\f030";
+}
+.fa-font:before {
+  content: "\f031";
+}
+.fa-bold:before {
+  content: "\f032";
+}
+.fa-italic:before {
+  content: "\f033";
+}
+.fa-text-height:before {
+  content: "\f034";
+}
+.fa-text-width:before {
+  content: "\f035";
+}
+.fa-align-left:before {
+  content: "\f036";
+}
+.fa-align-center:before {
+  content: "\f037";
+}
+.fa-align-right:before {
+  content: "\f038";
+}
+.fa-align-justify:before {
+  content: "\f039";
+}
+.fa-list:before {
+  content: "\f03a";
+}
+.fa-dedent:before,
+.fa-outdent:before {
+  content: "\f03b";
+}
+.fa-indent:before {
+  content: "\f03c";
+}
+.fa-video-camera:before {
+  content: "\f03d";
+}
+.fa-photo:before,
+.fa-image:before,
+.fa-picture-o:before {
+  content: "\f03e";
+}
+.fa-pencil:before {
+  content: "\f040";
+}
+.fa-map-marker:before {
+  content: "\f041";
+}
+.fa-adjust:before {
+  content: "\f042";
+}
+.fa-tint:before {
+  content: "\f043";
+}
+.fa-edit:before,
+.fa-pencil-square-o:before {
+  content: "\f044";
+}
+.fa-share-square-o:before {
+  content: "\f045";
+}
+.fa-check-square-o:before {
+  content: "\f046";
+}
+.fa-arrows:before {
+  content: "\f047";
+}
+.fa-step-backward:before {
+  content: "\f048";
+}
+.fa-fast-backward:before {
+  content: "\f049";
+}
+.fa-backward:before {
+  content: "\f04a";
+}
+.fa-play:before {
+  content: "\f04b";
+}
+.fa-pause:before {
+  content: "\f04c";
+}
+.fa-stop:before {
+  content: "\f04d";
+}
+.fa-forward:before {
+  content: "\f04e";
+}
+.fa-fast-forward:before {
+  content: "\f050";
+}
+.fa-step-forward:before {
+  content: "\f051";
+}
+.fa-eject:before {
+  content: "\f052";
+}
+.fa-chevron-left:before {
+  content: "\f053";
+}
+.fa-chevron-right:before {
+  content: "\f054";
+}
+.fa-plus-circle:before {
+  content: "\f055";
+}
+.fa-minus-circle:before {
+  content: "\f056";
+}
+.fa-times-circle:before {
+  content: "\f057";
+}
+.fa-check-circle:before {
+  content: "\f058";
+}
+.fa-question-circle:before {
+  content: "\f059";
+}
+.fa-info-circle:before {
+  content: "\f05a";
+}
+.fa-crosshairs:before {
+  content: "\f05b";
+}
+.fa-times-circle-o:before {
+  content: "\f05c";
+}
+.fa-check-circle-o:before {
+  content: "\f05d";
+}
+.fa-ban:before {
+  content: "\f05e";
+}
+.fa-arrow-left:before {
+  content: "\f060";
+}
+.fa-arrow-right:before {
+  content: "\f061";
+}
+.fa-arrow-up:before {
+  content: "\f062";
+}
+.fa-arrow-down:before {
+  content: "\f063";
+}
+.fa-mail-forward:before,
+.fa-share:before {
+  content: "\f064";
+}
+.fa-expand:before {
+  content: "\f065";
+}
+.fa-compress:before {
+  content: "\f066";
+}
+.fa-plus:before {
+  content: "\f067";
+}
+.fa-minus:before {
+  content: "\f068";
+}
+.fa-asterisk:before {
+  content: "\f069";
+}
+.fa-exclamation-circle:before {
+  content: "\f06a";
+}
+.fa-gift:before {
+  content: "\f06b";
+}
+.fa-leaf:before {
+  content: "\f06c";
+}
+.fa-fire:before {
+  content: "\f06d";
+}
+.fa-eye:before {
+  content: "\f06e";
+}
+.fa-eye-slash:before {
+  content: "\f070";
+}
+.fa-warning:before,
+.fa-exclamation-triangle:before {
+  content: "\f071";
+}
+.fa-plane:before {
+  content: "\f072";
+}
+.fa-calendar:before {
+  content: "\f073";
+}
+.fa-random:before {
+  content: "\f074";
+}
+.fa-comment:before {
+  content: "\f075";
+}
+.fa-magnet:before {
+  content: "\f076";
+}
+.fa-chevron-up:before {
+  content: "\f077";
+}
+.fa-chevron-down:before {
+  content: "\f078";
+}
+.fa-retweet:before {
+  content: "\f079";
+}
+.fa-shopping-cart:before {
+  content: "\f07a";
+}
+.fa-folder:before {
+  content: "\f07b";
+}
+.fa-folder-open:before {
+  content: "\f07c";
+}
+.fa-arrows-v:before {
+  content: "\f07d";
+}
+.fa-arrows-h:before {
+  content: "\f07e";
+}
+.fa-bar-chart-o:before,
+.fa-bar-chart:before {
+  content: "\f080";
+}
+.fa-twitter-square:before {
+  content: "\f081";
+}
+.fa-facebook-square:before {
+  content: "\f082";
+}
+.fa-camera-retro:before {
+  content: "\f083";
+}
+.fa-key:before {
+  content: "\f084";
+}
+.fa-gears:before,
+.fa-cogs:before {
+  content: "\f085";
+}
+.fa-comments:before {
+  content: "\f086";
+}
+.fa-thumbs-o-up:before {
+  content: "\f087";
+}
+.fa-thumbs-o-down:before {
+  content: "\f088";
+}
+.fa-star-half:before {
+  content: "\f089";
+}
+.fa-heart-o:before {
+  content: "\f08a";
+}
+.fa-sign-out:before {
+  content: "\f08b";
+}
+.fa-linkedin-square:before {
+  content: "\f08c";
+}
+.fa-thumb-tack:before {
+  content: "\f08d";
+}
+.fa-external-link:before {
+  content: "\f08e";
+}
+.fa-sign-in:before {
+  content: "\f090";
+}
+.fa-trophy:before {
+  content: "\f091";
+}
+.fa-github-square:before {
+  content: "\f092";
+}
+.fa-upload:before {
+  content: "\f093";
+}
+.fa-lemon-o:before {
+  content: "\f094";
+}
+.fa-phone:before {
+  content: "\f095";
+}
+.fa-square-o:before {
+  content: "\f096";
+}
+.fa-bookmark-o:before {
+  content: "\f097";
+}
+.fa-phone-square:before {
+  content: "\f098";
+}
+.fa-twitter:before {
+  content: "\f099";
+}
+.fa-facebook-f:before,
+.fa-facebook:before {
+  content: "\f09a";
+}
+.fa-github:before {
+  content: "\f09b";
+}
+.fa-unlock:before {
+  content: "\f09c";
+}
+.fa-credit-card:before {
+  content: "\f09d";
+}
+.fa-rss:before {
+  content: "\f09e";
+}
+.fa-hdd-o:before {
+  content: "\f0a0";
+}
+.fa-bullhorn:before {
+  content: "\f0a1";
+}
+.fa-bell:before {
+  content: "\f0f3";
+}
+.fa-certificate:before {
+  content: "\f0a3";
+}
+.fa-hand-o-right:before {
+  content: "\f0a4";
+}
+.fa-hand-o-left:before {
+  content: "\f0a5";
+}
+.fa-hand-o-up:before {
+  content: "\f0a6";
+}
+.fa-hand-o-down:before {
+  content: "\f0a7";
+}
+.fa-arrow-circle-left:before {
+  content: "\f0a8";
+}
+.fa-arrow-circle-right:before {
+  content: "\f0a9";
+}
+.fa-arrow-circle-up:before {
+  content: "\f0aa";
+}
+.fa-arrow-circle-down:before {
+  content: "\f0ab";
+}
+.fa-globe:before {
+  content: "\f0ac";
+}
+.fa-wrench:before {
+  content: "\f0ad";
+}
+.fa-tasks:before {
+  content: "\f0ae";
+}
+.fa-filter:before {
+  content: "\f0b0";
+}
+.fa-briefcase:before {
+  content: "\f0b1";
+}
+.fa-arrows-alt:before {
+  content: "\f0b2";
+}
+.fa-group:before,
+.fa-users:before {
+  content: "\f0c0";
+}
+.fa-chain:before,
+.fa-link:before {
+  content: "\f0c1";
+}
+.fa-cloud:before {
+  content: "\f0c2";
+}
+.fa-flask:before {
+  content: "\f0c3";
+}
+.fa-cut:before,
+.fa-scissors:before {
+  content: "\f0c4";
+}
+.fa-copy:before,
+.fa-files-o:before {
+  content: "\f0c5";
+}
+.fa-paperclip:before {
+  content: "\f0c6";
+}
+.fa-save:before,
+.fa-floppy-o:before {
+  content: "\f0c7";
+}
+.fa-square:before {
+  content: "\f0c8";
+}
+.fa-navicon:before,
+.fa-reorder:before,
+.fa-bars:before {
+  content: "\f0c9";
+}
+.fa-list-ul:before {
+  content: "\f0ca";
+}
+.fa-list-ol:before {
+  content: "\f0cb";
+}
+.fa-strikethrough:before {
+  content: "\f0cc";
+}
+.fa-underline:before {
+  content: "\f0cd";
+}
+.fa-table:before {
+  content: "\f0ce";
+}
+.fa-magic:before {
+  content: "\f0d0";
+}
+.fa-truck:before {
+  content: "\f0d1";
+}
+.fa-pinterest:before {
+  content: "\f0d2";
+}
+.fa-pinterest-square:before {
+  content: "\f0d3";
+}
+.fa-google-plus-square:before {
+  content: "\f0d4";
+}
+.fa-google-plus:before {
+  content: "\f0d5";
+}
+.fa-money:before {
+  content: "\f0d6";
+}
+.fa-caret-down:before {
+  content: "\f0d7";
+}
+.fa-caret-up:before {
+  content: "\f0d8";
+}
+.fa-caret-left:before {
+  content: "\f0d9";
+}
+.fa-caret-right:before {
+  content: "\f0da";
+}
+.fa-columns:before {
+  content: "\f0db";
+}
+.fa-unsorted:before,
+.fa-sort:before {
+  content: "\f0dc";
+}
+.fa-sort-down:before,
+.fa-sort-desc:before {
+  content: "\f0dd";
+}
+.fa-sort-up:before,
+.fa-sort-asc:before {
+  content: "\f0de";
+}
+.fa-envelope:before {
+  content: "\f0e0";
+}
+.fa-linkedin:before {
+  content: "\f0e1";
+}
+.fa-rotate-left:before,
+.fa-undo:before {
+  content: "\f0e2";
+}
+.fa-legal:before,
+.fa-gavel:before {
+  content: "\f0e3";
+}
+.fa-dashboard:before,
+.fa-tachometer:before {
+  content: "\f0e4";
+}
+.fa-comment-o:before {
+  content: "\f0e5";
+}
+.fa-comments-o:before {
+  content: "\f0e6";
+}
+.fa-flash:before,
+.fa-bolt:before {
+  content: "\f0e7";
+}
+.fa-sitemap:before {
+  content: "\f0e8";
+}
+.fa-umbrella:before {
+  content: "\f0e9";
+}
+.fa-paste:before,
+.fa-clipboard:before {
+  content: "\f0ea";
+}
+.fa-lightbulb-o:before {
+  content: "\f0eb";
+}
+.fa-exchange:before {
+  content: "\f0ec";
+}
+.fa-cloud-download:before {
+  content: "\f0ed";
+}
+.fa-cloud-upload:before {
+  content: "\f0ee";
+}
+.fa-user-md:before {
+  content: "\f0f0";
+}
+.fa-stethoscope:before {
+  content: "\f0f1";
+}
+.fa-suitcase:before {
+  content: "\f0f2";
+}
+.fa-bell-o:before {
+  content: "\f0a2";
+}
+.fa-coffee:before {
+  content: "\f0f4";
+}
+.fa-cutlery:before {
+  content: "\f0f5";
+}
+.fa-file-text-o:before {
+  content: "\f0f6";
+}
+.fa-building-o:before {
+  content: "\f0f7";
+}
+.fa-hospital-o:before {
+  content: "\f0f8";
+}
+.fa-ambulance:before {
+  content: "\f0f9";
+}
+.fa-medkit:before {
+  content: "\f0fa";
+}
+.fa-fighter-jet:before {
+  content: "\f0fb";
+}
+.fa-beer:before {
+  content: "\f0fc";
+}
+.fa-h-square:before {
+  content: "\f0fd";
+}
+.fa-plus-square:before {
+  content: "\f0fe";
+}
+.fa-angle-double-left:before {
+  content: "\f100";
+}
+.fa-angle-double-right:before {
+  content: "\f101";
+}
+.fa-angle-double-up:before {
+  content: "\f102";
+}
+.fa-angle-double-down:before {
+  content: "\f103";
+}
+.fa-angle-left:before {
+  content: "\f104";
+}
+.fa-angle-right:before {
+  content: "\f105";
+}
+.fa-angle-up:before {
+  content: "\f106";
+}
+.fa-angle-down:before {
+  content: "\f107";
+}
+.fa-desktop:before {
+  content: "\f108";
+}
+.fa-laptop:before {
+  content: "\f109";
+}
+.fa-tablet:before {
+  content: "\f10a";
+}
+.fa-mobile-phone:before,
+.fa-mobile:before {
+  content: "\f10b";
+}
+.fa-circle-o:before {
+  content: "\f10c";
+}
+.fa-quote-left:before {
+  content: "\f10d";
+}
+.fa-quote-right:before {
+  content: "\f10e";
+}
+.fa-spinner:before {
+  content: "\f110";
+}
+.fa-circle:before {
+  content: "\f111";
+}
+.fa-mail-reply:before,
+.fa-reply:before {
+  content: "\f112";
+}
+.fa-github-alt:before {
+  content: "\f113";
+}
+.fa-folder-o:before {
+  content: "\f114";
+}
+.fa-folder-open-o:before {
+  content: "\f115";
+}
+.fa-smile-o:before {
+  content: "\f118";
+}
+.fa-frown-o:before {
+  content: "\f119";
+}
+.fa-meh-o:before {
+  content: "\f11a";
+}
+.fa-gamepad:before {
+  content: "\f11b";
+}
+.fa-keyboard-o:before {
+  content: "\f11c";
+}
+.fa-flag-o:before {
+  content: "\f11d";
+}
+.fa-flag-checkered:before {
+  content: "\f11e";
+}
+.fa-terminal:before {
+  content: "\f120";
+}
+.fa-code:before {
+  content: "\f121";
+}
+.fa-mail-reply-all:before,
+.fa-reply-all:before {
+  content: "\f122";
+}
+.fa-star-half-empty:before,
+.fa-star-half-full:before,
+.fa-star-half-o:before {
+  content: "\f123";
+}
+.fa-location-arrow:before {
+  content: "\f124";
+}
+.fa-crop:before {
+  content: "\f125";
+}
+.fa-code-fork:before {
+  content: "\f126";
+}
+.fa-unlink:before,
+.fa-chain-broken:before {
+  content: "\f127";
+}
+.fa-question:before {
+  content: "\f128";
+}
+.fa-info:before {
+  content: "\f129";
+}
+.fa-exclamation:before {
+  content: "\f12a";
+}
+.fa-superscript:before {
+  content: "\f12b";
+}
+.fa-subscript:before {
+  content: "\f12c";
+}
+.fa-eraser:before {
+  content: "\f12d";
+}
+.fa-puzzle-piece:before {
+  content: "\f12e";
+}
+.fa-microphone:before {
+  content: "\f130";
+}
+.fa-microphone-slash:before {
+  content: "\f131";
+}
+.fa-shield:before {
+  content: "\f132";
+}
+.fa-calendar-o:before {
+  content: "\f133";
+}
+.fa-fire-extinguisher:before {
+  content: "\f134";
+}
+.fa-rocket:before {
+  content: "\f135";
+}
+.fa-maxcdn:before {
+  content: "\f136";
+}
+.fa-chevron-circle-left:before {
+  content: "\f137";
+}
+.fa-chevron-circle-right:before {
+  content: "\f138";
+}
+.fa-chevron-circle-up:before {
+  content: "\f139";
+}
+.fa-chevron-circle-down:before {
+  content: "\f13a";
+}
+.fa-html5:before {
+  content: "\f13b";
+}
+.fa-css3:before {
+  content: "\f13c";
+}
+.fa-anchor:before {
+  content: "\f13d";
+}
+.fa-unlock-alt:before {
+  content: "\f13e";
+}
+.fa-bullseye:before {
+  content: "\f140";
+}
+.fa-ellipsis-h:before {
+  content: "\f141";
+}
+.fa-ellipsis-v:before {
+  content: "\f142";
+}
+.fa-rss-square:before {
+  content: "\f143";
+}
+.fa-play-circle:before {
+  content: "\f144";
+}
+.fa-ticket:before {
+  content: "\f145";
+}
+.fa-minus-square:before {
+  content: "\f146";
+}
+.fa-minus-square-o:before {
+  content: "\f147";
+}
+.fa-level-up:before {
+  content: "\f148";
+}
+.fa-level-down:before {
+  content: "\f149";
+}
+.fa-check-square:before {
+  content: "\f14a";
+}
+.fa-pencil-square:before {
+  content: "\f14b";
+}
+.fa-external-link-square:before {
+  content: "\f14c";
+}
+.fa-share-square:before {
+  content: "\f14d";
+}
+.fa-compass:before {
+  content: "\f14e";
+}
+.fa-toggle-down:before,
+.fa-caret-square-o-down:before {
+  content: "\f150";
+}
+.fa-toggle-up:before,
+.fa-caret-square-o-up:before {
+  content: "\f151";
+}
+.fa-toggle-right:before,
+.fa-caret-square-o-right:before {
+  content: "\f152";
+}
+.fa-euro:before,
+.fa-eur:before {
+  content: "\f153";
+}
+.fa-gbp:before {
+  content: "\f154";
+}
+.fa-dollar:before,
+.fa-usd:before {
+  content: "\f155";
+}
+.fa-rupee:before,
+.fa-inr:before {
+  content: "\f156";
+}
+.fa-cny:before,
+.fa-rmb:before,
+.fa-yen:before,
+.fa-jpy:before {
+  content: "\f157";
+}
+.fa-ruble:before,
+.fa-rouble:before,
+.fa-rub:before {
+  content: "\f158";
+}
+.fa-won:before,
+.fa-krw:before {
+  content: "\f159";
+}
+.fa-bitcoin:before,
+.fa-btc:before {
+  content: "\f15a";
+}
+.fa-file:before {
+  content: "\f15b";
+}
+.fa-file-text:before {
+  content: "\f15c";
+}
+.fa-sort-alpha-asc:before {
+  content: "\f15d";
+}
+.fa-sort-alpha-desc:before {
+  content: "\f15e";
+}
+.fa-sort-amount-asc:before {
+  content: "\f160";
+}
+.fa-sort-amount-desc:before {
+  content: "\f161";
+}
+.fa-sort-numeric-asc:before {
+  content: "\f162";
+}
+.fa-sort-numeric-desc:before {
+  content: "\f163";
+}
+.fa-thumbs-up:before {
+  content: "\f164";
+}
+.fa-thumbs-down:before {
+  content: "\f165";
+}
+.fa-youtube-square:before {
+  content: "\f166";
+}
+.fa-youtube:before {
+  content: "\f167";
+}
+.fa-xing:before {
+  content: "\f168";
+}
+.fa-xing-square:before {
+  content: "\f169";
+}
+.fa-youtube-play:before {
+  content: "\f16a";
+}
+.fa-dropbox:before {
+  content: "\f16b";
+}
+.fa-stack-overflow:before {
+  content: "\f16c";
+}
+.fa-instagram:before {
+  content: "\f16d";
+}
+.fa-flickr:before {
+  content: "\f16e";
+}
+.fa-adn:before {
+  content: "\f170";
+}
+.fa-bitbucket:before {
+  content: "\f171";
+}
+.fa-bitbucket-square:before {
+  content: "\f172";
+}
+.fa-tumblr:before {
+  content: "\f173";
+}
+.fa-tumblr-square:before {
+  content: "\f174";
+}
+.fa-long-arrow-down:before {
+  content: "\f175";
+}
+.fa-long-arrow-up:before {
+  content: "\f176";
+}
+.fa-long-arrow-left:before {
+  content: "\f177";
+}
+.fa-long-arrow-right:before {
+  content: "\f178";
+}
+.fa-apple:before {
+  content: "\f179";
+}
+.fa-windows:before {
+  content: "\f17a";
+}
+.fa-android:before {
+  content: "\f17b";
+}
+.fa-linux:before {
+  content: "\f17c";
+}
+.fa-dribbble:before {
+  content: "\f17d";
+}
+.fa-skype:before {
+  content: "\f17e";
+}
+.fa-foursquare:before {
+  content: "\f180";
+}
+.fa-trello:before {
+  content: "\f181";
+}
+.fa-female:before {
+  content: "\f182";
+}
+.fa-male:before {
+  content: "\f183";
+}
+.fa-gittip:before,
+.fa-gratipay:before {
+  content: "\f184";
+}
+.fa-sun-o:before {
+  content: "\f185";
+}
+.fa-moon-o:before {
+  content: "\f186";
+}
+.fa-archive:before {
+  content: "\f187";
+}
+.fa-bug:before {
+  content: "\f188";
+}
+.fa-vk:before {
+  content: "\f189";
+}
+.fa-weibo:before {
+  content: "\f18a";
+}
+.fa-renren:before {
+  content: "\f18b";
+}
+.fa-pagelines:before {
+  content: "\f18c";
+}
+.fa-stack-exchange:before {
+  content: "\f18d";
+}
+.fa-arrow-circle-o-right:before {
+  content: "\f18e";
+}
+.fa-arrow-circle-o-left:before {
+  content: "\f190";
+}
+.fa-toggle-left:before,
+.fa-caret-square-o-left:before {
+  content: "\f191";
+}
+.fa-dot-circle-o:before {
+  content: "\f192";
+}
+.fa-wheelchair:before {
+  content: "\f193";
+}
+.fa-vimeo-square:before {
+  content: "\f194";
+}
+.fa-turkish-lira:before,
+.fa-try:before {
+  content: "\f195";
+}
+.fa-plus-square-o:before {
+  content: "\f196";
+}
+.fa-space-shuttle:before {
+  content: "\f197";
+}
+.fa-slack:before {
+  content: "\f198";
+}
+.fa-envelope-square:before {
+  content: "\f199";
+}
+.fa-wordpress:before {
+  content: "\f19a";
+}
+.fa-openid:before {
+  content: "\f19b";
+}
+.fa-institution:before,
+.fa-bank:before,
+.fa-university:before {
+  content: "\f19c";
+}
+.fa-mortar-board:before,
+.fa-graduation-cap:before {
+  content: "\f19d";
+}
+.fa-yahoo:before {
+  content: "\f19e";
+}
+.fa-google:before {
+  content: "\f1a0";
+}
+.fa-reddit:before {
+  content: "\f1a1";
+}
+.fa-reddit-square:before {
+  content: "\f1a2";
+}
+.fa-stumbleupon-circle:before {
+  content: "\f1a3";
+}
+.fa-stumbleupon:before {
+  content: "\f1a4";
+}
+.fa-delicious:before {
+  content: "\f1a5";
+}
+.fa-digg:before {
+  content: "\f1a6";
+}
+.fa-pied-piper:before {
+  content: "\f1a7";
+}
+.fa-pied-piper-alt:before {
+  content: "\f1a8";
+}
+.fa-drupal:before {
+  content: "\f1a9";
+}
+.fa-joomla:before {
+  content: "\f1aa";
+}
+.fa-language:before {
+  content: "\f1ab";
+}
+.fa-fax:before {
+  content: "\f1ac";
+}
+.fa-building:before {
+  content: "\f1ad";
+}
+.fa-child:before {
+  content: "\f1ae";
+}
+.fa-paw:before {
+  content: "\f1b0";
+}
+.fa-spoon:before {
+  content: "\f1b1";
+}
+.fa-cube:before {
+  content: "\f1b2";
+}
+.fa-cubes:before {
+  content: "\f1b3";
+}
+.fa-behance:before {
+  content: "\f1b4";
+}
+.fa-behance-square:before {
+  content: "\f1b5";
+}
+.fa-steam:before {
+  content: "\f1b6";
+}
+.fa-steam-square:before {
+  content: "\f1b7";
+}
+.fa-recycle:before {
+  content: "\f1b8";
+}
+.fa-automobile:before,
+.fa-car:before {
+  content: "\f1b9";
+}
+.fa-cab:before,
+.fa-taxi:before {
+  content: "\f1ba";
+}
+.fa-tree:before {
+  content: "\f1bb";
+}
+.fa-spotify:before {
+  content: "\f1bc";
+}
+.fa-deviantart:before {
+  content: "\f1bd";
+}
+.fa-soundcloud:before {
+  content: "\f1be";
+}
+.fa-database:before {
+  content: "\f1c0";
+}
+.fa-file-pdf-o:before {
+  content: "\f1c1";
+}
+.fa-file-word-o:before {
+  content: "\f1c2";
+}
+.fa-file-excel-o:before {
+  content: "\f1c3";
+}
+.fa-file-powerpoint-o:before {
+  content: "\f1c4";
+}
+.fa-file-photo-o:before,
+.fa-file-picture-o:before,
+.fa-file-image-o:before {
+  content: "\f1c5";
+}
+.fa-file-zip-o:before,
+.fa-file-archive-o:before {
+  content: "\f1c6";
+}
+.fa-file-sound-o:before,
+.fa-file-audio-o:before {
+  content: "\f1c7";
+}
+.fa-file-movie-o:before,
+.fa-file-video-o:before {
+  content: "\f1c8";
+}
+.fa-file-code-o:before {
+  content: "\f1c9";
+}
+.fa-vine:before {
+  content: "\f1ca";
+}
+.fa-codepen:before {
+  content: "\f1cb";
+}
+.fa-jsfiddle:before {
+  content: "\f1cc";
+}
+.fa-life-bouy:before,
+.fa-life-buoy:before,
+.fa-life-saver:before,
+.fa-support:before,
+.fa-life-ring:before {
+  content: "\f1cd";
+}
+.fa-circle-o-notch:before {
+  content: "\f1ce";
+}
+.fa-ra:before,
+.fa-rebel:before {
+  content: "\f1d0";
+}
+.fa-ge:before,
+.fa-empire:before {
+  content: "\f1d1";
+}
+.fa-git-square:before {
+  content: "\f1d2";
+}
+.fa-git:before {
+  content: "\f1d3";
+}
+.fa-hacker-news:before {
+  content: "\f1d4";
+}
+.fa-tencent-weibo:before {
+  content: "\f1d5";
+}
+.fa-qq:before {
+  content: "\f1d6";
+}
+.fa-wechat:before,
+.fa-weixin:before {
+  content: "\f1d7";
+}
+.fa-send:before,
+.fa-paper-plane:before {
+  content: "\f1d8";
+}
+.fa-send-o:before,
+.fa-paper-plane-o:before {
+  content: "\f1d9";
+}
+.fa-history:before {
+  content: "\f1da";
+}
+.fa-genderless:before,
+.fa-circle-thin:before {
+  content: "\f1db";
+}
+.fa-header:before {
+  content: "\f1dc";
+}
+.fa-paragraph:before {
+  content: "\f1dd";
+}
+.fa-sliders:before {
+  content: "\f1de";
+}
+.fa-share-alt:before {
+  content: "\f1e0";
+}
+.fa-share-alt-square:before {
+  content: "\f1e1";
+}
+.fa-bomb:before {
+  content: "\f1e2";
+}
+.fa-soccer-ball-o:before,
+.fa-futbol-o:before {
+  content: "\f1e3";
+}
+.fa-tty:before {
+  content: "\f1e4";
+}
+.fa-binoculars:before {
+  content: "\f1e5";
+}
+.fa-plug:before {
+  content: "\f1e6";
+}
+.fa-slideshare:before {
+  content: "\f1e7";
+}
+.fa-twitch:before {
+  content: "\f1e8";
+}
+.fa-yelp:before {
+  content: "\f1e9";
+}
+.fa-newspaper-o:before {
+  content: "\f1ea";
+}
+.fa-wifi:before {
+  content: "\f1eb";
+}
+.fa-calculator:before {
+  content: "\f1ec";
+}
+.fa-paypal:before {
+  content: "\f1ed";
+}
+.fa-google-wallet:before {
+  content: "\f1ee";
+}
+.fa-cc-visa:before {
+  content: "\f1f0";
+}
+.fa-cc-mastercard:before {
+  content: "\f1f1";
+}
+.fa-cc-discover:before {
+  content: "\f1f2";
+}
+.fa-cc-amex:before {
+  content: "\f1f3";
+}
+.fa-cc-paypal:before {
+  content: "\f1f4";
+}
+.fa-cc-stripe:before {
+  content: "\f1f5";
+}
+.fa-bell-slash:before {
+  content: "\f1f6";
+}
+.fa-bell-slash-o:before {
+  content: "\f1f7";
+}
+.fa-trash:before {
+  content: "\f1f8";
+}
+.fa-copyright:before {
+  content: "\f1f9";
+}
+.fa-at:before {
+  content: "\f1fa";
+}
+.fa-eyedropper:before {
+  content: "\f1fb";
+}
+.fa-paint-brush:before {
+  content: "\f1fc";
+}
+.fa-birthday-cake:before {
+  content: "\f1fd";
+}
+.fa-area-chart:before {
+  content: "\f1fe";
+}
+.fa-pie-chart:before {
+  content: "\f200";
+}
+.fa-line-chart:before {
+  content: "\f201";
+}
+.fa-lastfm:before {
+  content: "\f202";
+}
+.fa-lastfm-square:before {
+  content: "\f203";
+}
+.fa-toggle-off:before {
+  content: "\f204";
+}
+.fa-toggle-on:before {
+  content: "\f205";
+}
+.fa-bicycle:before {
+  content: "\f206";
+}
+.fa-bus:before {
+  content: "\f207";
+}
+.fa-ioxhost:before {
+  content: "\f208";
+}
+.fa-angellist:before {
+  content: "\f209";
+}
+.fa-cc:before {
+  content: "\f20a";
+}
+.fa-shekel:before,
+.fa-sheqel:before,
+.fa-ils:before {
+  content: "\f20b";
+}
+.fa-meanpath:before {
+  content: "\f20c";
+}
+.fa-buysellads:before {
+  content: "\f20d";
+}
+.fa-connectdevelop:before {
+  content: "\f20e";
+}
+.fa-dashcube:before {
+  content: "\f210";
+}
+.fa-forumbee:before {
+  content: "\f211";
+}
+.fa-leanpub:before {
+  content: "\f212";
+}
+.fa-sellsy:before {
+  content: "\f213";
+}
+.fa-shirtsinbulk:before {
+  content: "\f214";
+}
+.fa-simplybuilt:before {
+  content: "\f215";
+}
+.fa-skyatlas:before {
+  content: "\f216";
+}
+.fa-cart-plus:before {
+  content: "\f217";
+}
+.fa-cart-arrow-down:before {
+  content: "\f218";
+}
+.fa-diamond:before {
+  content: "\f219";
+}
+.fa-ship:before {
+  content: "\f21a";
+}
+.fa-user-secret:before {
+  content: "\f21b";
+}
+.fa-motorcycle:before {
+  content: "\f21c";
+}
+.fa-street-view:before {
+  content: "\f21d";
+}
+.fa-heartbeat:before {
+  content: "\f21e";
+}
+.fa-venus:before {
+  content: "\f221";
+}
+.fa-mars:before {
+  content: "\f222";
+}
+.fa-mercury:before {
+  content: "\f223";
+}
+.fa-transgender:before {
+  content: "\f224";
+}
+.fa-transgender-alt:before {
+  content: "\f225";
+}
+.fa-venus-double:before {
+  content: "\f226";
+}
+.fa-mars-double:before {
+  content: "\f227";
+}
+.fa-venus-mars:before {
+  content: "\f228";
+}
+.fa-mars-stroke:before {
+  content: "\f229";
+}
+.fa-mars-stroke-v:before {
+  content: "\f22a";
+}
+.fa-mars-stroke-h:before {
+  content: "\f22b";
+}
+.fa-neuter:before {
+  content: "\f22c";
+}
+.fa-facebook-official:before {
+  content: "\f230";
+}
+.fa-pinterest-p:before {
+  content: "\f231";
+}
+.fa-whatsapp:before {
+  content: "\f232";
+}
+.fa-server:before {
+  content: "\f233";
+}
+.fa-user-plus:before {
+  content: "\f234";
+}
+.fa-user-times:before {
+  content: "\f235";
+}
+.fa-hotel:before,
+.fa-bed:before {
+  content: "\f236";
+}
+.fa-viacoin:before {
+  content: "\f237";
+}
+.fa-train:before {
+  content: "\f238";
+}
+.fa-subway:before {
+  content: "\f239";
+}
+.fa-medium:before {
+  content: "\f23a";
+}
+
+/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */
+html {
+  font-family: sans-serif;
+  -ms-text-size-adjust: 100%;
+  -webkit-text-size-adjust: 100%;
+}
+body {
+  margin: 0;
+}
+article,
+aside,
+details,
+figcaption,
+figure,
+footer,
+header,
+hgroup,
+main,
+menu,
+nav,
+section,
+summary {
+  display: block;
+}
+audio,
+canvas,
+progress,
+video {
+  display: inline-block;
+  vertical-align: baseline;
+}
+audio:not([controls]) {
+  display: none;
+  height: 0;
+}
+[hidden],
+template {
+  display: none;
+}
+a {
+  background-color: transparent;
+}
+a:active,
+a:hover {
+  outline: 0;
+}
+abbr[title] {
+  border-bottom: 1px dotted;
+}
+b,
+strong {
+  font-weight: bold;
+}
+dfn {
+  font-style: italic;
+}
+h1 {
+  font-size: 2em;
+  margin: 0.67em 0;
+}
+mark {
+  background: #ff0;
+  color: #000;
+}
+small {
+  font-size: 80%;
+}
+sub,
+sup {
+  font-size: 75%;
+  line-height: 0;
+  position: relative;
+  vertical-align: baseline;
+}
+sup {
+  top: -0.5em;
+}
+sub {
+  bottom: -0.25em;
+}
+img {
+  border: 0;
+}
+svg:not(:root) {
+  overflow: hidden;
+}
+figure {
+  margin: 1em 40px;
+}
+hr {
+  box-sizing: content-box;
+  height: 0;
+}
+pre {
+  overflow: auto;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: monospace, monospace;
+  font-size: 1em;
+}
+button,
+input,
+optgroup,
+select,
+textarea {
+  color: inherit;
+  font: inherit;
+  margin: 0;
+}
+button {
+  overflow: visible;
+}
+button,
+select {
+  text-transform: none;
+}
+button,
+html input[type="button"],
+input[type="reset"],
+input[type="submit"] {
+  -webkit-appearance: button;
+  cursor: pointer;
+}
+button[disabled],
+html input[disabled] {
+  cursor: default;
+}
+button::-moz-focus-inner,
+input::-moz-focus-inner {
+  border: 0;
+  padding: 0;
+}
+input {
+  line-height: normal;
+}
+input[type="checkbox"],
+input[type="radio"] {
+  box-sizing: border-box;
+  padding: 0;
+}
+input[type="number"]::-webkit-inner-spin-button,
+input[type="number"]::-webkit-outer-spin-button {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: textfield;
+  box-sizing: content-box;
+}
+input[type="search"]::-webkit-search-cancel-button,
+input[type="search"]::-webkit-search-decoration {
+  -webkit-appearance: none;
+}
+fieldset {
+  border: 1px solid #c0c0c0;
+  margin: 0 2px;
+  padding: 0.35em 0.625em 0.75em;
+}
+legend {
+  border: 0;
+  padding: 0;
+}
+textarea {
+  overflow: auto;
+}
+optgroup {
+  font-weight: bold;
+}
+table {
+  border-collapse: collapse;
+  border-spacing: 0;
+}
+td,
+th {
+  padding: 0;
+}
+/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */
+@media print {
+  *,
+  *:before,
+  *:after {
+    background: transparent !important;
+    color: #000 !important;
+    box-shadow: none !important;
+    text-shadow: none !important;
+  }
+  a,
+  a:visited {
+    text-decoration: underline;
+  }
+  a[href]:after {
+    content: " (" attr(href) ")";
+  }
+  abbr[title]:after {
+    content: " (" attr(title) ")";
+  }
+  a[href^="#"]:after,
+  a[href^="javascript:"]:after {
+    content: "";
+  }
+  pre,
+  blockquote {
+    border: 1px solid #999;
+    page-break-inside: avoid;
+  }
+  thead {
+    display: table-header-group;
+  }
+  tr,
+  img {
+    page-break-inside: avoid;
+  }
+  img {
+    max-width: 100% !important;
+  }
+  p,
+  h2,
+  h3 {
+    orphans: 3;
+    widows: 3;
+  }
+  h2,
+  h3 {
+    page-break-after: avoid;
+  }
+  .navbar {
+    display: none;
+  }
+  .btn > .caret,
+  .dropup > .btn > .caret {
+    border-top-color: #000 !important;
+  }
+  .label {
+    border: 1px solid #000;
+  }
+  .table {
+    border-collapse: collapse !important;
+  }
+  .table td,
+  .table th {
+    background-color: #fff !important;
+  }
+  .table-bordered th,
+  .table-bordered td {
+    border: 1px solid #ddd !important;
+  }
+}
+@font-face {
+  font-family: 'Glyphicons Halflings';
+  src: url('../fonts/glyphicons-halflings-regular.eot');
+  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
+}
+.glyphicon {
+  position: relative;
+  top: 1px;
+  display: inline-block;
+  font-family: 'Glyphicons Halflings';
+  font-style: normal;
+  font-weight: normal;
+  line-height: 1;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+}
+.glyphicon-asterisk:before {
+  content: "\2a";
+}
+.glyphicon-plus:before {
+  content: "\2b";
+}
+.glyphicon-euro:before,
+.glyphicon-eur:before {
+  content: "\20ac";
+}
+.glyphicon-minus:before {
+  content: "\2212";
+}
+.glyphicon-cloud:before {
+  content: "\2601";
+}
+.glyphicon-envelope:before {
+  content: "\2709";
+}
+.glyphicon-pencil:before {
+  content: "\270f";
+}
+.glyphicon-glass:before {
+  content: "\e001";
+}
+.glyphicon-music:before {
+  content: "\e002";
+}
+.glyphicon-search:before {
+  content: "\e003";
+}
+.glyphicon-heart:before {
+  content: "\e005";
+}
+.glyphicon-star:before {
+  content: "\e006";
+}
+.glyphicon-star-empty:before {
+  content: "\e007";
+}
+.glyphicon-user:before {
+  content: "\e008";
+}
+.glyphicon-film:before {
+  content: "\e009";
+}
+.glyphicon-th-large:before {
+  content: "\e010";
+}
+.glyphicon-th:before {
+  content: "\e011";
+}
+.glyphicon-th-list:before {
+  content: "\e012";
+}
+.glyphicon-ok:before {
+  content: "\e013";
+}
+.glyphicon-remove:before {
+  content: "\e014";
+}
+.glyphicon-zoom-in:before {
+  content: "\e015";
+}
+.glyphicon-zoom-out:before {
+  content: "\e016";
+}
+.glyphicon-off:before {
+  content: "\e017";
+}
+.glyphicon-signal:before {
+  content: "\e018";
+}
+.glyphicon-cog:before {
+  content: "\e019";
+}
+.glyphicon-trash:before {
+  content: "\e020";
+}
+.glyphicon-home:before {
+  content: "\e021";
+}
+.glyphicon-file:before {
+  content: "\e022";
+}
+.glyphicon-time:before {
+  content: "\e023";
+}
+.glyphicon-road:before {
+  content: "\e024";
+}
+.glyphicon-download-alt:before {
+  content: "\e025";
+}
+.glyphicon-download:before {
+  content: "\e026";
+}
+.glyphicon-upload:before {
+  content: "\e027";
+}
+.glyphicon-inbox:before {
+  content: "\e028";
+}
+.glyphicon-play-circle:before {
+  content: "\e029";
+}
+.glyphicon-repeat:before {
+  content: "\e030";
+}
+.glyphicon-refresh:before {
+  content: "\e031";
+}
+.glyphicon-list-alt:before {
+  content: "\e032";
+}
+.glyphicon-lock:before {
+  content: "\e033";
+}
+.glyphicon-flag:before {
+  content: "\e034";
+}
+.glyphicon-headphones:before {
+  content: "\e035";
+}
+.glyphicon-volume-off:before {
+  content: "\e036";
+}
+.glyphicon-volume-down:before {
+  content: "\e037";
+}
+.glyphicon-volume-up:before {
+  content: "\e038";
+}
+.glyphicon-qrcode:before {
+  content: "\e039";
+}
+.glyphicon-barcode:before {
+  content: "\e040";
+}
+.glyphicon-tag:before {
+  content: "\e041";
+}
+.glyphicon-tags:before {
+  content: "\e042";
+}
+.glyphicon-book:before {
+  content: "\e043";
+}
+.glyphicon-bookmark:before {
+  content: "\e044";
+}
+.glyphicon-print:before {
+  content: "\e045";
+}
+.glyphicon-camera:before {
+  content: "\e046";
+}
+.glyphicon-font:before {
+  content: "\e047";
+}
+.glyphicon-bold:before {
+  content: "\e048";
+}
+.glyphicon-italic:before {
+  content: "\e049";
+}
+.glyphicon-text-height:before {
+  content: "\e050";
+}
+.glyphicon-text-width:before {
+  content: "\e051";
+}
+.glyphicon-align-left:before {
+  content: "\e052";
+}
+.glyphicon-align-center:before {
+  content: "\e053";
+}
+.glyphicon-align-right:before {
+  content: "\e054";
+}
+.glyphicon-align-justify:before {
+  content: "\e055";
+}
+.glyphicon-list:before {
+  content: "\e056";
+}
+.glyphicon-indent-left:before {
+  content: "\e057";
+}
+.glyphicon-indent-right:before {
+  content: "\e058";
+}
+.glyphicon-facetime-video:before {
+  content: "\e059";
+}
+.glyphicon-picture:before {
+  content: "\e060";
+}
+.glyphicon-map-marker:before {
+  content: "\e062";
+}
+.glyphicon-adjust:before {
+  content: "\e063";
+}
+.glyphicon-tint:before {
+  content: "\e064";
+}
+.glyphicon-edit:before {
+  content: "\e065";
+}
+.glyphicon-share:before {
+  content: "\e066";
+}
+.glyphicon-check:before {
+  content: "\e067";
+}
+.glyphicon-move:before {
+  content: "\e068";
+}
+.glyphicon-step-backward:before {
+  content: "\e069";
+}
+.glyphicon-fast-backward:before {
+  content: "\e070";
+}
+.glyphicon-backward:before {
+  content: "\e071";
+}
+.glyphicon-play:before {
+  content: "\e072";
+}
+.glyphicon-pause:before {
+  content: "\e073";
+}
+.glyphicon-stop:before {
+  content: "\e074";
+}
+.glyphicon-forward:before {
+  content: "\e075";
+}
+.glyphicon-fast-forward:before {
+  content: "\e076";
+}
+.glyphicon-step-forward:before {
+  content: "\e077";
+}
+.glyphicon-eject:before {
+  content: "\e078";
+}
+.glyphicon-chevron-left:before {
+  content: "\e079";
+}
+.glyphicon-chevron-right:before {
+  content: "\e080";
+}
+.glyphicon-plus-sign:before {
+  content: "\e081";
+}
+.glyphicon-minus-sign:before {
+  content: "\e082";
+}
+.glyphicon-remove-sign:before {
+  content: "\e083";
+}
+.glyphicon-ok-sign:before {
+  content: "\e084";
+}
+.glyphicon-question-sign:before {
+  content: "\e085";
+}
+.glyphicon-info-sign:before {
+  content: "\e086";
+}
+.glyphicon-screenshot:before {
+  content: "\e087";
+}
+.glyphicon-remove-circle:before {
+  content: "\e088";
+}
+.glyphicon-ok-circle:before {
+  content: "\e089";
+}
+.glyphicon-ban-circle:before {
+  content: "\e090";
+}
+.glyphicon-arrow-left:before {
+  content: "\e091";
+}
+.glyphicon-arrow-right:before {
+  content: "\e092";
+}
+.glyphicon-arrow-up:before {
+  content: "\e093";
+}
+.glyphicon-arrow-down:before {
+  content: "\e094";
+}
+.glyphicon-share-alt:before {
+  content: "\e095";
+}
+.glyphicon-resize-full:before {
+  content: "\e096";
+}
+.glyphicon-resize-small:before {
+  content: "\e097";
+}
+.glyphicon-exclamation-sign:before {
+  content: "\e101";
+}
+.glyphicon-gift:before {
+  content: "\e102";
+}
+.glyphicon-leaf:before {
+  content: "\e103";
+}
+.glyphicon-fire:before {
+  content: "\e104";
+}
+.glyphicon-eye-open:before {
+  content: "\e105";
+}
+.glyphicon-eye-close:before {
+  content: "\e106";
+}
+.glyphicon-warning-sign:before {
+  content: "\e107";
+}
+.glyphicon-plane:before {
+  content: "\e108";
+}
+.glyphicon-calendar:before {
+  content: "\e109";
+}
+.glyphicon-random:before {
+  content: "\e110";
+}
+.glyphicon-comment:before {
+  content: "\e111";
+}
+.glyphicon-magnet:before {
+  content: "\e112";
+}
+.glyphicon-chevron-up:before {
+  content: "\e113";
+}
+.glyphicon-chevron-down:before {
+  content: "\e114";
+}
+.glyphicon-retweet:before {
+  content: "\e115";
+}
+.glyphicon-shopping-cart:before {
+  content: "\e116";
+}
+.glyphicon-folder-close:before {
+  content: "\e117";
+}
+.glyphicon-folder-open:before {
+  content: "\e118";
+}
+.glyphicon-resize-vertical:before {
+  content: "\e119";
+}
+.glyphicon-resize-horizontal:before {
+  content: "\e120";
+}
+.glyphicon-hdd:before {
+  content: "\e121";
+}
+.glyphicon-bullhorn:before {
+  content: "\e122";
+}
+.glyphicon-bell:before {
+  content: "\e123";
+}
+.glyphicon-certificate:before {
+  content: "\e124";
+}
+.glyphicon-thumbs-up:before {
+  content: "\e125";
+}
+.glyphicon-thumbs-down:before {
+  content: "\e126";
+}
+.glyphicon-hand-right:before {
+  content: "\e127";
+}
+.glyphicon-hand-left:before {
+  content: "\e128";
+}
+.glyphicon-hand-up:before {
+  content: "\e129";
+}
+.glyphicon-hand-down:before {
+  content: "\e130";
+}
+.glyphicon-circle-arrow-right:before {
+  content: "\e131";
+}
+.glyphicon-circle-arrow-left:before {
+  content: "\e132";
+}
+.glyphicon-circle-arrow-up:before {
+  content: "\e133";
+}
+.glyphicon-circle-arrow-down:before {
+  content: "\e134";
+}
+.glyphicon-globe:before {
+  content: "\e135";
+}
+.glyphicon-wrench:before {
+  content: "\e136";
+}
+.glyphicon-tasks:before {
+  content: "\e137";
+}
+.glyphicon-filter:before {
+  content: "\e138";
+}
+.glyphicon-briefcase:before {
+  content: "\e139";
+}
+.glyphicon-fullscreen:before {
+  content: "\e140";
+}
+.glyphicon-dashboard:before {
+  content: "\e141";
+}
+.glyphicon-paperclip:before {
+  content: "\e142";
+}
+.glyphicon-heart-empty:before {
+  content: "\e143";
+}
+.glyphicon-link:before {
+  content: "\e144";
+}
+.glyphicon-phone:before {
+  content: "\e145";
+}
+.glyphicon-pushpin:before {
+  content: "\e146";
+}
+.glyphicon-usd:before {
+  content: "\e148";
+}
+.glyphicon-gbp:before {
+  content: "\e149";
+}
+.glyphicon-sort:before {
+  content: "\e150";
+}
+.glyphicon-sort-by-alphabet:before {
+  content: "\e151";
+}
+.glyphicon-sort-by-alphabet-alt:before {
+  content: "\e152";
+}
+.glyphicon-sort-by-order:before {
+  content: "\e153";
+}
+.glyphicon-sort-by-order-alt:before {
+  content: "\e154";
+}
+.glyphicon-sort-by-attributes:before {
+  content: "\e155";
+}
+.glyphicon-sort-by-attributes-alt:before {
+  content: "\e156";
+}
+.glyphicon-unchecked:before {
+  content: "\e157";
+}
+.glyphicon-expand:before {
+  content: "\e158";
+}
+.glyphicon-collapse-down:before {
+  content: "\e159";
+}
+.glyphicon-collapse-up:before {
+  content: "\e160";
+}
+.glyphicon-log-in:before {
+  content: "\e161";
+}
+.glyphicon-flash:before {
+  content: "\e162";
+}
+.glyphicon-log-out:before {
+  content: "\e163";
+}
+.glyphicon-new-window:before {
+  content: "\e164";
+}
+.glyphicon-record:before {
+  content: "\e165";
+}
+.glyphicon-save:before {
+  content: "\e166";
+}
+.glyphicon-open:before {
+  content: "\e167";
+}
+.glyphicon-saved:before {
+  content: "\e168";
+}
+.glyphicon-import:before {
+  content: "\e169";
+}
+.glyphicon-export:before {
+  content: "\e170";
+}
+.glyphicon-send:before {
+  content: "\e171";
+}
+.glyphicon-floppy-disk:before {
+  content: "\e172";
+}
+.glyphicon-floppy-saved:before {
+  content: "\e173";
+}
+.glyphicon-floppy-remove:before {
+  content: "\e174";
+}
+.glyphicon-floppy-save:before {
+  content: "\e175";
+}
+.glyphicon-floppy-open:before {
+  content: "\e176";
+}
+.glyphicon-credit-card:before {
+  content: "\e177";
+}
+.glyphicon-transfer:before {
+  content: "\e178";
+}
+.glyphicon-cutlery:before {
+  content: "\e179";
+}
+.glyphicon-header:before {
+  content: "\e180";
+}
+.glyphicon-compressed:before {
+  content: "\e181";
+}
+.glyphicon-earphone:before {
+  content: "\e182";
+}
+.glyphicon-phone-alt:before {
+  content: "\e183";
+}
+.glyphicon-tower:before {
+  content: "\e184";
+}
+.glyphicon-stats:before {
+  content: "\e185";
+}
+.glyphicon-sd-video:before {
+  content: "\e186";
+}
+.glyphicon-hd-video:before {
+  content: "\e187";
+}
+.glyphicon-subtitles:before {
+  content: "\e188";
+}
+.glyphicon-sound-stereo:before {
+  content: "\e189";
+}
+.glyphicon-sound-dolby:before {
+  content: "\e190";
+}
+.glyphicon-sound-5-1:before {
+  content: "\e191";
+}
+.glyphicon-sound-6-1:before {
+  content: "\e192";
+}
+.glyphicon-sound-7-1:before {
+  content: "\e193";
+}
+.glyphicon-copyright-mark:before {
+  content: "\e194";
+}
+.glyphicon-registration-mark:before {
+  content: "\e195";
+}
+.glyphicon-cloud-download:before {
+  content: "\e197";
+}
+.glyphicon-cloud-upload:before {
+  content: "\e198";
+}
+.glyphicon-tree-conifer:before {
+  content: "\e199";
+}
+.glyphicon-tree-deciduous:before {
+  content: "\e200";
+}
+.glyphicon-cd:before {
+  content: "\e201";
+}
+.glyphicon-save-file:before {
+  content: "\e202";
+}
+.glyphicon-open-file:before {
+  content: "\e203";
+}
+.glyphicon-level-up:before {
+  content: "\e204";
+}
+.glyphicon-copy:before {
+  content: "\e205";
+}
+.glyphicon-paste:before {
+  content: "\e206";
+}
+.glyphicon-alert:before {
+  content: "\e209";
+}
+.glyphicon-equalizer:before {
+  content: "\e210";
+}
+.glyphicon-king:before {
+  content: "\e211";
+}
+.glyphicon-queen:before {
+  content: "\e212";
+}
+.glyphicon-pawn:before {
+  content: "\e213";
+}
+.glyphicon-bishop:before {
+  content: "\e214";
+}
+.glyphicon-knight:before {
+  content: "\e215";
+}
+.glyphicon-baby-formula:before {
+  content: "\e216";
+}
+.glyphicon-tent:before {
+  content: "\26fa";
+}
+.glyphicon-blackboard:before {
+  content: "\e218";
+}
+.glyphicon-bed:before {
+  content: "\e219";
+}
+.glyphicon-apple:before {
+  content: "\f8ff";
+}
+.glyphicon-erase:before {
+  content: "\e221";
+}
+.glyphicon-hourglass:before {
+  content: "\231b";
+}
+.glyphicon-lamp:before {
+  content: "\e223";
+}
+.glyphicon-duplicate:before {
+  content: "\e224";
+}
+.glyphicon-piggy-bank:before {
+  content: "\e225";
+}
+.glyphicon-scissors:before {
+  content: "\e226";
+}
+.glyphicon-bitcoin:before {
+  content: "\e227";
+}
+.glyphicon-btc:before {
+  content: "\e227";
+}
+.glyphicon-xbt:before {
+  content: "\e227";
+}
+.glyphicon-yen:before {
+  content: "\00a5";
+}
+.glyphicon-jpy:before {
+  content: "\00a5";
+}
+.glyphicon-ruble:before {
+  content: "\20bd";
+}
+.glyphicon-rub:before {
+  content: "\20bd";
+}
+.glyphicon-scale:before {
+  content: "\e230";
+}
+.glyphicon-ice-lolly:before {
+  content: "\e231";
+}
+.glyphicon-ice-lolly-tasted:before {
+  content: "\e232";
+}
+.glyphicon-education:before {
+  content: "\e233";
+}
+.glyphicon-option-horizontal:before {
+  content: "\e234";
+}
+.glyphicon-option-vertical:before {
+  content: "\e235";
+}
+.glyphicon-menu-hamburger:before {
+  content: "\e236";
+}
+.glyphicon-modal-window:before {
+  content: "\e237";
+}
+.glyphicon-oil:before {
+  content: "\e238";
+}
+.glyphicon-grain:before {
+  content: "\e239";
+}
+.glyphicon-sunglasses:before {
+  content: "\e240";
+}
+.glyphicon-text-size:before {
+  content: "\e241";
+}
+.glyphicon-text-color:before {
+  content: "\e242";
+}
+.glyphicon-text-background:before {
+  content: "\e243";
+}
+.glyphicon-object-align-top:before {
+  content: "\e244";
+}
+.glyphicon-object-align-bottom:before {
+  content: "\e245";
+}
+.glyphicon-object-align-horizontal:before {
+  content: "\e246";
+}
+.glyphicon-object-align-left:before {
+  content: "\e247";
+}
+.glyphicon-object-align-vertical:before {
+  content: "\e248";
+}
+.glyphicon-object-align-right:before {
+  content: "\e249";
+}
+.glyphicon-triangle-right:before {
+  content: "\e250";
+}
+.glyphicon-triangle-left:before {
+  content: "\e251";
+}
+.glyphicon-triangle-bottom:before {
+  content: "\e252";
+}
+.glyphicon-triangle-top:before {
+  content: "\e253";
+}
+.glyphicon-console:before {
+  content: "\e254";
+}
+.glyphicon-superscript:before {
+  content: "\e255";
+}
+.glyphicon-subscript:before {
+  content: "\e256";
+}
+.glyphicon-menu-left:before {
+  content: "\e257";
+}
+.glyphicon-menu-right:before {
+  content: "\e258";
+}
+.glyphicon-menu-down:before {
+  content: "\e259";
+}
+.glyphicon-menu-up:before {
+  content: "\e260";
+}
+* {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+*:before,
+*:after {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+html {
+  font-size: 10px;
+  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
+}
+body {
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #333333;
+  background-color: #fff;
+}
+input,
+button,
+select,
+textarea {
+  font-family: inherit;
+  font-size: inherit;
+  line-height: inherit;
+}
+a {
+  color: #158cba;
+  text-decoration: none;
+}
+a:hover,
+a:focus {
+  color: #0d5875;
+  text-decoration: underline;
+}
+a:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+figure {
+  margin: 0;
+}
+img {
+  vertical-align: middle;
+}
+.img-responsive,
+.thumbnail > img,
+.thumbnail a > img,
+.carousel-inner > .item > img,
+.carousel-inner > .item > a > img {
+  display: block;
+  max-width: 100%;
+  height: auto;
+}
+.img-rounded {
+  border-radius: 6px;
+}
+.img-thumbnail {
+  padding: 4px;
+  line-height: 1.42857143;
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 4px;
+  -webkit-transition: all 0.2s ease-in-out;
+  -o-transition: all 0.2s ease-in-out;
+  transition: all 0.2s ease-in-out;
+  display: inline-block;
+  max-width: 100%;
+  height: auto;
+}
+.img-circle {
+  border-radius: 50%;
+}
+hr {
+  margin-top: 20px;
+  margin-bottom: 20px;
+  border: 0;
+  border-top: 1px solid #eeeeee;
+}
+.sr-only {
+  position: absolute;
+  width: 1px;
+  height: 1px;
+  margin: -1px;
+  padding: 0;
+  overflow: hidden;
+  clip: rect(0, 0, 0, 0);
+  border: 0;
+}
+.sr-only-focusable:active,
+.sr-only-focusable:focus {
+  position: static;
+  width: auto;
+  height: auto;
+  margin: 0;
+  overflow: visible;
+  clip: auto;
+}
+[role="button"] {
+  cursor: pointer;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6,
+.h1,
+.h2,
+.h3,
+.h4,
+.h5,
+.h6 {
+  font-family: inherit;
+  font-weight: 500;
+  line-height: 1.1;
+  color: inherit;
+}
+h1 small,
+h2 small,
+h3 small,
+h4 small,
+h5 small,
+h6 small,
+.h1 small,
+.h2 small,
+.h3 small,
+.h4 small,
+.h5 small,
+.h6 small,
+h1 .small,
+h2 .small,
+h3 .small,
+h4 .small,
+h5 .small,
+h6 .small,
+.h1 .small,
+.h2 .small,
+.h3 .small,
+.h4 .small,
+.h5 .small,
+.h6 .small {
+  font-weight: normal;
+  line-height: 1;
+  color: #777777;
+}
+h1,
+.h1,
+h2,
+.h2,
+h3,
+.h3 {
+  margin-top: 20px;
+  margin-bottom: 10px;
+}
+h1 small,
+.h1 small,
+h2 small,
+.h2 small,
+h3 small,
+.h3 small,
+h1 .small,
+.h1 .small,
+h2 .small,
+.h2 .small,
+h3 .small,
+.h3 .small {
+  font-size: 65%;
+}
+h4,
+.h4,
+h5,
+.h5,
+h6,
+.h6 {
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+h4 small,
+.h4 small,
+h5 small,
+.h5 small,
+h6 small,
+.h6 small,
+h4 .small,
+.h4 .small,
+h5 .small,
+.h5 .small,
+h6 .small,
+.h6 .small {
+  font-size: 75%;
+}
+h1,
+.h1 {
+  font-size: 28px;
+}
+h2,
+.h2 {
+  font-size: 23px;
+}
+h3,
+.h3 {
+  font-size: 21px;
+}
+h4,
+.h4 {
+  font-size: 18px;
+}
+h5,
+.h5 {
+  font-size: 14px;
+}
+h6,
+.h6 {
+  font-size: 12px;
+}
+p {
+  margin: 0 0 10px;
+}
+.lead {
+  margin-bottom: 20px;
+  font-size: 16px;
+  font-weight: 300;
+  line-height: 1.4;
+}
+@media (min-width: 768px) {
+  .lead {
+    font-size: 21px;
+  }
+}
+small,
+.small {
+  font-size: 85%;
+}
+mark,
+.mark {
+  background-color: #fcf8e3;
+  padding: .2em;
+}
+.text-left {
+  text-align: left;
+}
+.text-right {
+  text-align: right;
+}
+.text-center {
+  text-align: center;
+}
+.text-justify {
+  text-align: justify;
+}
+.text-nowrap {
+  white-space: nowrap;
+}
+.text-lowercase {
+  text-transform: lowercase;
+}
+.text-uppercase {
+  text-transform: uppercase;
+}
+.text-capitalize {
+  text-transform: capitalize;
+}
+.text-muted {
+  color: #777777;
+}
+.text-primary {
+  color: #158cba;
+}
+a.text-primary:hover,
+a.text-primary:focus {
+  color: #106a8c;
+}
+.text-success {
+  color: #3c763d;
+}
+a.text-success:hover,
+a.text-success:focus {
+  color: #2b542c;
+}
+.text-info {
+  color: #31708f;
+}
+a.text-info:hover,
+a.text-info:focus {
+  color: #245269;
+}
+.text-warning {
+  color: #8a6d3b;
+}
+a.text-warning:hover,
+a.text-warning:focus {
+  color: #66512c;
+}
+.text-danger {
+  color: #a94442;
+}
+a.text-danger:hover,
+a.text-danger:focus {
+  color: #843534;
+}
+.bg-primary {
+  color: #fff;
+  background-color: #158cba;
+}
+a.bg-primary:hover,
+a.bg-primary:focus {
+  background-color: #106a8c;
+}
+.bg-success {
+  background-color: #dff0d8;
+}
+a.bg-success:hover,
+a.bg-success:focus {
+  background-color: #c1e2b3;
+}
+.bg-info {
+  background-color: #d9edf7;
+}
+a.bg-info:hover,
+a.bg-info:focus {
+  background-color: #afd9ee;
+}
+.bg-warning {
+  background-color: #fcf8e3;
+}
+a.bg-warning:hover,
+a.bg-warning:focus {
+  background-color: #f7ecb5;
+}
+.bg-danger {
+  background-color: #f2dede;
+}
+a.bg-danger:hover,
+a.bg-danger:focus {
+  background-color: #e4b9b9;
+}
+.page-header {
+  padding-bottom: 9px;
+  margin: 40px 0 20px;
+  border-bottom: 1px solid #eeeeee;
+}
+ul,
+ol {
+  margin-top: 0;
+  margin-bottom: 10px;
+}
+ul ul,
+ol ul,
+ul ol,
+ol ol {
+  margin-bottom: 0;
+}
+.list-unstyled {
+  padding-left: 0;
+  list-style: none;
+}
+.list-inline {
+  padding-left: 0;
+  list-style: none;
+  margin-left: -5px;
+}
+.list-inline > li {
+  display: inline-block;
+  padding-left: 5px;
+  padding-right: 5px;
+}
+dl {
+  margin-top: 0;
+  margin-bottom: 20px;
+}
+dt,
+dd {
+  line-height: 1.42857143;
+}
+dt {
+  font-weight: bold;
+}
+dd {
+  margin-left: 0;
+}
+@media (min-width: 768px) {
+  .dl-horizontal dt {
+    float: left;
+    width: 160px;
+    clear: left;
+    text-align: right;
+    overflow: hidden;
+    text-overflow: ellipsis;
+    white-space: nowrap;
+  }
+  .dl-horizontal dd {
+    margin-left: 180px;
+  }
+}
+abbr[title],
+abbr[data-original-title] {
+  cursor: help;
+  border-bottom: 1px dotted #777777;
+}
+.initialism {
+  font-size: 90%;
+  text-transform: uppercase;
+}
+blockquote {
+  padding: 10px 20px;
+  margin: 0 0 20px;
+  font-size: 17.5px;
+  border-left: 5px solid #eeeeee;
+}
+blockquote p:last-child,
+blockquote ul:last-child,
+blockquote ol:last-child {
+  margin-bottom: 0;
+}
+blockquote footer,
+blockquote small,
+blockquote .small {
+  display: block;
+  font-size: 80%;
+  line-height: 1.42857143;
+  color: #777777;
+}
+blockquote footer:before,
+blockquote small:before,
+blockquote .small:before {
+  content: '\2014 \00A0';
+}
+.blockquote-reverse,
+blockquote.pull-right {
+  padding-right: 15px;
+  padding-left: 0;
+  border-right: 5px solid #eeeeee;
+  border-left: 0;
+  text-align: right;
+}
+.blockquote-reverse footer:before,
+blockquote.pull-right footer:before,
+.blockquote-reverse small:before,
+blockquote.pull-right small:before,
+.blockquote-reverse .small:before,
+blockquote.pull-right .small:before {
+  content: '';
+}
+.blockquote-reverse footer:after,
+blockquote.pull-right footer:after,
+.blockquote-reverse small:after,
+blockquote.pull-right small:after,
+.blockquote-reverse .small:after,
+blockquote.pull-right .small:after {
+  content: '\00A0 \2014';
+}
+address {
+  margin-bottom: 20px;
+  font-style: normal;
+  line-height: 1.42857143;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
+}
+code {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #c7254e;
+  background-color: #f9f2f4;
+  border-radius: 4px;
+}
+kbd {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #fff;
+  background-color: #333;
+  border-radius: 3px;
+  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);
+}
+kbd kbd {
+  padding: 0;
+  font-size: 100%;
+  font-weight: bold;
+  box-shadow: none;
+}
+pre {
+  display: block;
+  padding: 9.5px;
+  margin: 0 0 10px;
+  font-size: 13px;
+  line-height: 1.42857143;
+  word-break: break-all;
+  word-wrap: break-word;
+  color: #333333;
+  background-color: #f5f5f5;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+}
+pre code {
+  padding: 0;
+  font-size: inherit;
+  color: inherit;
+  white-space: pre-wrap;
+  background-color: transparent;
+  border-radius: 0;
+}
+.pre-scrollable {
+  max-height: 340px;
+  overflow-y: scroll;
+}
+.container {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+@media (min-width: 768px) {
+  .container {
+    width: 750px;
+  }
+}
+@media (min-width: 992px) {
+  .container {
+    width: 970px;
+  }
+}
+@media (min-width: 1200px) {
+  .container {
+    width: 1170px;
+  }
+}
+.container-fluid {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.row {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
+  position: relative;
+  min-height: 1px;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
+  float: left;
+}
+.col-xs-12 {
+  width: 100%;
+}
+.col-xs-11 {
+  width: 91.66666667%;
+}
+.col-xs-10 {
+  width: 83.33333333%;
+}
+.col-xs-9 {
+  width: 75%;
+}
+.col-xs-8 {
+  width: 66.66666667%;
+}
+.col-xs-7 {
+  width: 58.33333333%;
+}
+.col-xs-6 {
+  width: 50%;
+}
+.col-xs-5 {
+  width: 41.66666667%;
+}
+.col-xs-4 {
+  width: 33.33333333%;
+}
+.col-xs-3 {
+  width: 25%;
+}
+.col-xs-2 {
+  width: 16.66666667%;
+}
+.col-xs-1 {
+  width: 8.33333333%;
+}
+.col-xs-pull-12 {
+  right: 100%;
+}
+.col-xs-pull-11 {
+  right: 91.66666667%;
+}
+.col-xs-pull-10 {
+  right: 83.33333333%;
+}
+.col-xs-pull-9 {
+  right: 75%;
+}
+.col-xs-pull-8 {
+  right: 66.66666667%;
+}
+.col-xs-pull-7 {
+  right: 58.33333333%;
+}
+.col-xs-pull-6 {
+  right: 50%;
+}
+.col-xs-pull-5 {
+  right: 41.66666667%;
+}
+.col-xs-pull-4 {
+  right: 33.33333333%;
+}
+.col-xs-pull-3 {
+  right: 25%;
+}
+.col-xs-pull-2 {
+  right: 16.66666667%;
+}
+.col-xs-pull-1 {
+  right: 8.33333333%;
+}
+.col-xs-pull-0 {
+  right: auto;
+}
+.col-xs-push-12 {
+  left: 100%;
+}
+.col-xs-push-11 {
+  left: 91.66666667%;
+}
+.col-xs-push-10 {
+  left: 83.33333333%;
+}
+.col-xs-push-9 {
+  left: 75%;
+}
+.col-xs-push-8 {
+  left: 66.66666667%;
+}
+.col-xs-push-7 {
+  left: 58.33333333%;
+}
+.col-xs-push-6 {
+  left: 50%;
+}
+.col-xs-push-5 {
+  left: 41.66666667%;
+}
+.col-xs-push-4 {
+  left: 33.33333333%;
+}
+.col-xs-push-3 {
+  left: 25%;
+}
+.col-xs-push-2 {
+  left: 16.66666667%;
+}
+.col-xs-push-1 {
+  left: 8.33333333%;
+}
+.col-xs-push-0 {
+  left: auto;
+}
+.col-xs-offset-12 {
+  margin-left: 100%;
+}
+.col-xs-offset-11 {
+  margin-left: 91.66666667%;
+}
+.col-xs-offset-10 {
+  margin-left: 83.33333333%;
+}
+.col-xs-offset-9 {
+  margin-left: 75%;
+}
+.col-xs-offset-8 {
+  margin-left: 66.66666667%;
+}
+.col-xs-offset-7 {
+  margin-left: 58.33333333%;
+}
+.col-xs-offset-6 {
+  margin-left: 50%;
+}
+.col-xs-offset-5 {
+  margin-left: 41.66666667%;
+}
+.col-xs-offset-4 {
+  margin-left: 33.33333333%;
+}
+.col-xs-offset-3 {
+  margin-left: 25%;
+}
+.col-xs-offset-2 {
+  margin-left: 16.66666667%;
+}
+.col-xs-offset-1 {
+  margin-left: 8.33333333%;
+}
+.col-xs-offset-0 {
+  margin-left: 0%;
+}
+@media (min-width: 768px) {
+  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
+    float: left;
+  }
+  .col-sm-12 {
+    width: 100%;
+  }
+  .col-sm-11 {
+    width: 91.66666667%;
+  }
+  .col-sm-10 {
+    width: 83.33333333%;
+  }
+  .col-sm-9 {
+    width: 75%;
+  }
+  .col-sm-8 {
+    width: 66.66666667%;
+  }
+  .col-sm-7 {
+    width: 58.33333333%;
+  }
+  .col-sm-6 {
+    width: 50%;
+  }
+  .col-sm-5 {
+    width: 41.66666667%;
+  }
+  .col-sm-4 {
+    width: 33.33333333%;
+  }
+  .col-sm-3 {
+    width: 25%;
+  }
+  .col-sm-2 {
+    width: 16.66666667%;
+  }
+  .col-sm-1 {
+    width: 8.33333333%;
+  }
+  .col-sm-pull-12 {
+    right: 100%;
+  }
+  .col-sm-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-sm-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-sm-pull-9 {
+    right: 75%;
+  }
+  .col-sm-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-sm-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-sm-pull-6 {
+    right: 50%;
+  }
+  .col-sm-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-sm-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-sm-pull-3 {
+    right: 25%;
+  }
+  .col-sm-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-sm-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-sm-pull-0 {
+    right: auto;
+  }
+  .col-sm-push-12 {
+    left: 100%;
+  }
+  .col-sm-push-11 {
+    left: 91.66666667%;
+  }
+  .col-sm-push-10 {
+    left: 83.33333333%;
+  }
+  .col-sm-push-9 {
+    left: 75%;
+  }
+  .col-sm-push-8 {
+    left: 66.66666667%;
+  }
+  .col-sm-push-7 {
+    left: 58.33333333%;
+  }
+  .col-sm-push-6 {
+    left: 50%;
+  }
+  .col-sm-push-5 {
+    left: 41.66666667%;
+  }
+  .col-sm-push-4 {
+    left: 33.33333333%;
+  }
+  .col-sm-push-3 {
+    left: 25%;
+  }
+  .col-sm-push-2 {
+    left: 16.66666667%;
+  }
+  .col-sm-push-1 {
+    left: 8.33333333%;
+  }
+  .col-sm-push-0 {
+    left: auto;
+  }
+  .col-sm-offset-12 {
+    margin-left: 100%;
+  }
+  .col-sm-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-sm-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-sm-offset-9 {
+    margin-left: 75%;
+  }
+  .col-sm-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-sm-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-sm-offset-6 {
+    margin-left: 50%;
+  }
+  .col-sm-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-sm-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-sm-offset-3 {
+    margin-left: 25%;
+  }
+  .col-sm-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-sm-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-sm-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 992px) {
+  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
+    float: left;
+  }
+  .col-md-12 {
+    width: 100%;
+  }
+  .col-md-11 {
+    width: 91.66666667%;
+  }
+  .col-md-10 {
+    width: 83.33333333%;
+  }
+  .col-md-9 {
+    width: 75%;
+  }
+  .col-md-8 {
+    width: 66.66666667%;
+  }
+  .col-md-7 {
+    width: 58.33333333%;
+  }
+  .col-md-6 {
+    width: 50%;
+  }
+  .col-md-5 {
+    width: 41.66666667%;
+  }
+  .col-md-4 {
+    width: 33.33333333%;
+  }
+  .col-md-3 {
+    width: 25%;
+  }
+  .col-md-2 {
+    width: 16.66666667%;
+  }
+  .col-md-1 {
+    width: 8.33333333%;
+  }
+  .col-md-pull-12 {
+    right: 100%;
+  }
+  .col-md-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-md-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-md-pull-9 {
+    right: 75%;
+  }
+  .col-md-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-md-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-md-pull-6 {
+    right: 50%;
+  }
+  .col-md-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-md-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-md-pull-3 {
+    right: 25%;
+  }
+  .col-md-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-md-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-md-pull-0 {
+    right: auto;
+  }
+  .col-md-push-12 {
+    left: 100%;
+  }
+  .col-md-push-11 {
+    left: 91.66666667%;
+  }
+  .col-md-push-10 {
+    left: 83.33333333%;
+  }
+  .col-md-push-9 {
+    left: 75%;
+  }
+  .col-md-push-8 {
+    left: 66.66666667%;
+  }
+  .col-md-push-7 {
+    left: 58.33333333%;
+  }
+  .col-md-push-6 {
+    left: 50%;
+  }
+  .col-md-push-5 {
+    left: 41.66666667%;
+  }
+  .col-md-push-4 {
+    left: 33.33333333%;
+  }
+  .col-md-push-3 {
+    left: 25%;
+  }
+  .col-md-push-2 {
+    left: 16.66666667%;
+  }
+  .col-md-push-1 {
+    left: 8.33333333%;
+  }
+  .col-md-push-0 {
+    left: auto;
+  }
+  .col-md-offset-12 {
+    margin-left: 100%;
+  }
+  .col-md-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-md-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-md-offset-9 {
+    margin-left: 75%;
+  }
+  .col-md-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-md-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-md-offset-6 {
+    margin-left: 50%;
+  }
+  .col-md-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-md-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-md-offset-3 {
+    margin-left: 25%;
+  }
+  .col-md-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-md-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-md-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 1200px) {
+  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
+    float: left;
+  }
+  .col-lg-12 {
+    width: 100%;
+  }
+  .col-lg-11 {
+    width: 91.66666667%;
+  }
+  .col-lg-10 {
+    width: 83.33333333%;
+  }
+  .col-lg-9 {
+    width: 75%;
+  }
+  .col-lg-8 {
+    width: 66.66666667%;
+  }
+  .col-lg-7 {
+    width: 58.33333333%;
+  }
+  .col-lg-6 {
+    width: 50%;
+  }
+  .col-lg-5 {
+    width: 41.66666667%;
+  }
+  .col-lg-4 {
+    width: 33.33333333%;
+  }
+  .col-lg-3 {
+    width: 25%;
+  }
+  .col-lg-2 {
+    width: 16.66666667%;
+  }
+  .col-lg-1 {
+    width: 8.33333333%;
+  }
+  .col-lg-pull-12 {
+    right: 100%;
+  }
+  .col-lg-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-lg-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-lg-pull-9 {
+    right: 75%;
+  }
+  .col-lg-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-lg-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-lg-pull-6 {
+    right: 50%;
+  }
+  .col-lg-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-lg-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-lg-pull-3 {
+    right: 25%;
+  }
+  .col-lg-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-lg-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-lg-pull-0 {
+    right: auto;
+  }
+  .col-lg-push-12 {
+    left: 100%;
+  }
+  .col-lg-push-11 {
+    left: 91.66666667%;
+  }
+  .col-lg-push-10 {
+    left: 83.33333333%;
+  }
+  .col-lg-push-9 {
+    left: 75%;
+  }
+  .col-lg-push-8 {
+    left: 66.66666667%;
+  }
+  .col-lg-push-7 {
+    left: 58.33333333%;
+  }
+  .col-lg-push-6 {
+    left: 50%;
+  }
+  .col-lg-push-5 {
+    left: 41.66666667%;
+  }
+  .col-lg-push-4 {
+    left: 33.33333333%;
+  }
+  .col-lg-push-3 {
+    left: 25%;
+  }
+  .col-lg-push-2 {
+    left: 16.66666667%;
+  }
+  .col-lg-push-1 {
+    left: 8.33333333%;
+  }
+  .col-lg-push-0 {
+    left: auto;
+  }
+  .col-lg-offset-12 {
+    margin-left: 100%;
+  }
+  .col-lg-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-lg-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-lg-offset-9 {
+    margin-left: 75%;
+  }
+  .col-lg-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-lg-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-lg-offset-6 {
+    margin-left: 50%;
+  }
+  .col-lg-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-lg-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-lg-offset-3 {
+    margin-left: 25%;
+  }
+  .col-lg-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-lg-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-lg-offset-0 {
+    margin-left: 0%;
+  }
+}
+table {
+  background-color: transparent;
+}
+caption {
+  padding-top: 8px;
+  padding-bottom: 8px;
+  color: #777777;
+  text-align: left;
+}
+th {
+  text-align: left;
+}
+.table {
+  width: 100%;
+  max-width: 100%;
+  margin-bottom: 20px;
+}
+.table > thead > tr > th,
+.table > tbody > tr > th,
+.table > tfoot > tr > th,
+.table > thead > tr > td,
+.table > tbody > tr > td,
+.table > tfoot > tr > td {
+  padding: 8px;
+  line-height: 1.42857143;
+  vertical-align: top;
+  border-top: 1px solid #ddd;
+}
+.table > thead > tr > th {
+  vertical-align: bottom;
+  border-bottom: 2px solid #ddd;
+}
+.table > caption + thead > tr:first-child > th,
+.table > colgroup + thead > tr:first-child > th,
+.table > thead:first-child > tr:first-child > th,
+.table > caption + thead > tr:first-child > td,
+.table > colgroup + thead > tr:first-child > td,
+.table > thead:first-child > tr:first-child > td {
+  border-top: 0;
+}
+.table > tbody + tbody {
+  border-top: 2px solid #ddd;
+}
+.table .table {
+  background-color: #fff;
+}
+.table-condensed > thead > tr > th,
+.table-condensed > tbody > tr > th,
+.table-condensed > tfoot > tr > th,
+.table-condensed > thead > tr > td,
+.table-condensed > tbody > tr > td,
+.table-condensed > tfoot > tr > td {
+  padding: 5px;
+}
+.table-bordered {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > tbody > tr > th,
+.table-bordered > tfoot > tr > th,
+.table-bordered > thead > tr > td,
+.table-bordered > tbody > tr > td,
+.table-bordered > tfoot > tr > td {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > thead > tr > td {
+  border-bottom-width: 2px;
+}
+.table-striped > tbody > tr:nth-of-type(odd) {
+  background-color: #f9f9f9;
+}
+.table-hover > tbody > tr:hover {
+  background-color: #f5f5f5;
+}
+table col[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-column;
+}
+table td[class*="col-"],
+table th[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-cell;
+}
+.table > thead > tr > td.active,
+.table > tbody > tr > td.active,
+.table > tfoot > tr > td.active,
+.table > thead > tr > th.active,
+.table > tbody > tr > th.active,
+.table > tfoot > tr > th.active,
+.table > thead > tr.active > td,
+.table > tbody > tr.active > td,
+.table > tfoot > tr.active > td,
+.table > thead > tr.active > th,
+.table > tbody > tr.active > th,
+.table > tfoot > tr.active > th {
+  background-color: #f5f5f5;
+}
+.table-hover > tbody > tr > td.active:hover,
+.table-hover > tbody > tr > th.active:hover,
+.table-hover > tbody > tr.active:hover > td,
+.table-hover > tbody > tr:hover > .active,
+.table-hover > tbody > tr.active:hover > th {
+  background-color: #e8e8e8;
+}
+.table > thead > tr > td.success,
+.table > tbody > tr > td.success,
+.table > tfoot > tr > td.success,
+.table > thead > tr > th.success,
+.table > tbody > tr > th.success,
+.table > tfoot > tr > th.success,
+.table > thead > tr.success > td,
+.table > tbody > tr.success > td,
+.table > tfoot > tr.success > td,
+.table > thead > tr.success > th,
+.table > tbody > tr.success > th,
+.table > tfoot > tr.success > th {
+  background-color: #dff0d8;
+}
+.table-hover > tbody > tr > td.success:hover,
+.table-hover > tbody > tr > th.success:hover,
+.table-hover > tbody > tr.success:hover > td,
+.table-hover > tbody > tr:hover > .success,
+.table-hover > tbody > tr.success:hover > th {
+  background-color: #d0e9c6;
+}
+.table > thead > tr > td.info,
+.table > tbody > tr > td.info,
+.table > tfoot > tr > td.info,
+.table > thead > tr > th.info,
+.table > tbody > tr > th.info,
+.table > tfoot > tr > th.info,
+.table > thead > tr.info > td,
+.table > tbody > tr.info > td,
+.table > tfoot > tr.info > td,
+.table > thead > tr.info > th,
+.table > tbody > tr.info > th,
+.table > tfoot > tr.info > th {
+  background-color: #d9edf7;
+}
+.table-hover > tbody > tr > td.info:hover,
+.table-hover > tbody > tr > th.info:hover,
+.table-hover > tbody > tr.info:hover > td,
+.table-hover > tbody > tr:hover > .info,
+.table-hover > tbody > tr.info:hover > th {
+  background-color: #c4e3f3;
+}
+.table > thead > tr > td.warning,
+.table > tbody > tr > td.warning,
+.table > tfoot > tr > td.warning,
+.table > thead > tr > th.warning,
+.table > tbody > tr > th.warning,
+.table > tfoot > tr > th.warning,
+.table > thead > tr.warning > td,
+.table > tbody > tr.warning > td,
+.table > tfoot > tr.warning > td,
+.table > thead > tr.warning > th,
+.table > tbody > tr.warning > th,
+.table > tfoot > tr.warning > th {
+  background-color: #fcf8e3;
+}
+.table-hover > tbody > tr > td.warning:hover,
+.table-hover > tbody > tr > th.warning:hover,
+.table-hover > tbody > tr.warning:hover > td,
+.table-hover > tbody > tr:hover > .warning,
+.table-hover > tbody > tr.warning:hover > th {
+  background-color: #faf2cc;
+}
+.table > thead > tr > td.danger,
+.table > tbody > tr > td.danger,
+.table > tfoot > tr > td.danger,
+.table > thead > tr > th.danger,
+.table > tbody > tr > th.danger,
+.table > tfoot > tr > th.danger,
+.table > thead > tr.danger > td,
+.table > tbody > tr.danger > td,
+.table > tfoot > tr.danger > td,
+.table > thead > tr.danger > th,
+.table > tbody > tr.danger > th,
+.table > tfoot > tr.danger > th {
+  background-color: #f2dede;
+}
+.table-hover > tbody > tr > td.danger:hover,
+.table-hover > tbody > tr > th.danger:hover,
+.table-hover > tbody > tr.danger:hover > td,
+.table-hover > tbody > tr:hover > .danger,
+.table-hover > tbody > tr.danger:hover > th {
+  background-color: #ebcccc;
+}
+.table-responsive {
+  overflow-x: auto;
+  min-height: 0.01%;
+}
+@media screen and (max-width: 767px) {
+  .table-responsive {
+    width: 100%;
+    margin-bottom: 15px;
+    overflow-y: hidden;
+    -ms-overflow-style: -ms-autohiding-scrollbar;
+    border: 1px solid #ddd;
+  }
+  .table-responsive > .table {
+    margin-bottom: 0;
+  }
+  .table-responsive > .table > thead > tr > th,
+  .table-responsive > .table > tbody > tr > th,
+  .table-responsive > .table > tfoot > tr > th,
+  .table-responsive > .table > thead > tr > td,
+  .table-responsive > .table > tbody > tr > td,
+  .table-responsive > .table > tfoot > tr > td {
+    white-space: nowrap;
+  }
+  .table-responsive > .table-bordered {
+    border: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:first-child,
+  .table-responsive > .table-bordered > tbody > tr > th:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
+  .table-responsive > .table-bordered > thead > tr > td:first-child,
+  .table-responsive > .table-bordered > tbody > tr > td:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
+    border-left: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:last-child,
+  .table-responsive > .table-bordered > tbody > tr > th:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
+  .table-responsive > .table-bordered > thead > tr > td:last-child,
+  .table-responsive > .table-bordered > tbody > tr > td:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
+    border-right: 0;
+  }
+  .table-responsive > .table-bordered > tbody > tr:last-child > th,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
+  .table-responsive > .table-bordered > tbody > tr:last-child > td,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
+    border-bottom: 0;
+  }
+}
+fieldset {
+  padding: 0;
+  margin: 0;
+  border: 0;
+  min-width: 0;
+}
+legend {
+  display: block;
+  width: 100%;
+  padding: 0;
+  margin-bottom: 20px;
+  font-size: 21px;
+  line-height: inherit;
+  color: #333333;
+  border: 0;
+  border-bottom: 1px solid #e5e5e5;
+}
+label {
+  display: inline-block;
+  max-width: 100%;
+  margin-bottom: 5px;
+  font-weight: bold;
+}
+input[type="search"] {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+input[type="radio"],
+input[type="checkbox"] {
+  margin: 4px 0 0;
+  margin-top: 1px \9;
+  line-height: normal;
+}
+input[type="file"] {
+  display: block;
+}
+input[type="range"] {
+  display: block;
+  width: 100%;
+}
+select[multiple],
+select[size] {
+  height: auto;
+}
+input[type="file"]:focus,
+input[type="radio"]:focus,
+input[type="checkbox"]:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+output {
+  display: block;
+  padding-top: 7px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555555;
+}
+.form-control {
+  display: block;
+  width: 100%;
+  height: 34px;
+  padding: 6px 12px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555555;
+  background-color: #fff;
+  background-image: none;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+  -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+  transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+}
+.form-control:focus {
+  border-color: #66afe9;
+  outline: 0;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+  box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+}
+.form-control::-moz-placeholder {
+  color: #999;
+  opacity: 1;
+}
+.form-control:-ms-input-placeholder {
+  color: #999;
+}
+.form-control::-webkit-input-placeholder {
+  color: #999;
+}
+.form-control[disabled],
+.form-control[readonly],
+fieldset[disabled] .form-control {
+  background-color: #eeeeee;
+  opacity: 1;
+}
+.form-control[disabled],
+fieldset[disabled] .form-control {
+  cursor: not-allowed;
+}
+textarea.form-control {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: none;
+}
+@media screen and (-webkit-min-device-pixel-ratio: 0) {
+  input[type="date"].form-control,
+  input[type="time"].form-control,
+  input[type="datetime-local"].form-control,
+  input[type="month"].form-control {
+    line-height: 34px;
+  }
+  input[type="date"].input-sm,
+  input[type="time"].input-sm,
+  input[type="datetime-local"].input-sm,
+  input[type="month"].input-sm,
+  .input-group-sm input[type="date"],
+  .input-group-sm input[type="time"],
+  .input-group-sm input[type="datetime-local"],
+  .input-group-sm input[type="month"] {
+    line-height: 30px;
+  }
+  input[type="date"].input-lg,
+  input[type="time"].input-lg,
+  input[type="datetime-local"].input-lg,
+  input[type="month"].input-lg,
+  .input-group-lg input[type="date"],
+  .input-group-lg input[type="time"],
+  .input-group-lg input[type="datetime-local"],
+  .input-group-lg input[type="month"] {
+    line-height: 46px;
+  }
+}
+.form-group {
+  margin-bottom: 15px;
+}
+.radio,
+.checkbox {
+  position: relative;
+  display: block;
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+.radio label,
+.checkbox label {
+  min-height: 20px;
+  padding-left: 20px;
+  margin-bottom: 0;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio input[type="radio"],
+.radio-inline input[type="radio"],
+.checkbox input[type="checkbox"],
+.checkbox-inline input[type="checkbox"] {
+  position: absolute;
+  margin-left: -20px;
+  margin-top: 4px \9;
+}
+.radio + .radio,
+.checkbox + .checkbox {
+  margin-top: -5px;
+}
+.radio-inline,
+.checkbox-inline {
+  position: relative;
+  display: inline-block;
+  padding-left: 20px;
+  margin-bottom: 0;
+  vertical-align: middle;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio-inline + .radio-inline,
+.checkbox-inline + .checkbox-inline {
+  margin-top: 0;
+  margin-left: 10px;
+}
+input[type="radio"][disabled],
+input[type="checkbox"][disabled],
+input[type="radio"].disabled,
+input[type="checkbox"].disabled,
+fieldset[disabled] input[type="radio"],
+fieldset[disabled] input[type="checkbox"] {
+  cursor: not-allowed;
+}
+.radio-inline.disabled,
+.checkbox-inline.disabled,
+fieldset[disabled] .radio-inline,
+fieldset[disabled] .checkbox-inline {
+  cursor: not-allowed;
+}
+.radio.disabled label,
+.checkbox.disabled label,
+fieldset[disabled] .radio label,
+fieldset[disabled] .checkbox label {
+  cursor: not-allowed;
+}
+.form-control-static {
+  padding-top: 7px;
+  padding-bottom: 7px;
+  margin-bottom: 0;
+  min-height: 34px;
+}
+.form-control-static.input-lg,
+.form-control-static.input-sm {
+  padding-left: 0;
+  padding-right: 0;
+}
+.input-sm {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+select.input-sm {
+  height: 30px;
+  line-height: 30px;
+}
+textarea.input-sm,
+select[multiple].input-sm {
+  height: auto;
+}
+.form-group-sm .form-control {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+.form-group-sm select.form-control {
+  height: 30px;
+  line-height: 30px;
+}
+.form-group-sm textarea.form-control,
+.form-group-sm select[multiple].form-control {
+  height: auto;
+}
+.form-group-sm .form-control-static {
+  height: 30px;
+  min-height: 32px;
+  padding: 6px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+}
+.input-lg {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+  border-radius: 6px;
+}
+select.input-lg {
+  height: 46px;
+  line-height: 46px;
+}
+textarea.input-lg,
+select[multiple].input-lg {
+  height: auto;
+}
+.form-group-lg .form-control {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+  border-radius: 6px;
+}
+.form-group-lg select.form-control {
+  height: 46px;
+  line-height: 46px;
+}
+.form-group-lg textarea.form-control,
+.form-group-lg select[multiple].form-control {
+  height: auto;
+}
+.form-group-lg .form-control-static {
+  height: 46px;
+  min-height: 38px;
+  padding: 11px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+}
+.has-feedback {
+  position: relative;
+}
+.has-feedback .form-control {
+  padding-right: 42.5px;
+}
+.form-control-feedback {
+  position: absolute;
+  top: 0;
+  right: 0;
+  z-index: 2;
+  display: block;
+  width: 34px;
+  height: 34px;
+  line-height: 34px;
+  text-align: center;
+  pointer-events: none;
+}
+.input-lg + .form-control-feedback,
+.input-group-lg + .form-control-feedback,
+.form-group-lg .form-control + .form-control-feedback {
+  width: 46px;
+  height: 46px;
+  line-height: 46px;
+}
+.input-sm + .form-control-feedback,
+.input-group-sm + .form-control-feedback,
+.form-group-sm .form-control + .form-control-feedback {
+  width: 30px;
+  height: 30px;
+  line-height: 30px;
+}
+.has-success .help-block,
+.has-success .control-label,
+.has-success .radio,
+.has-success .checkbox,
+.has-success .radio-inline,
+.has-success .checkbox-inline,
+.has-success.radio label,
+.has-success.checkbox label,
+.has-success.radio-inline label,
+.has-success.checkbox-inline label {
+  color: #3c763d;
+}
+.has-success .form-control {
+  border-color: #3c763d;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-success .form-control:focus {
+  border-color: #2b542c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
+}
+.has-success .input-group-addon {
+  color: #3c763d;
+  border-color: #3c763d;
+  background-color: #dff0d8;
+}
+.has-success .form-control-feedback {
+  color: #3c763d;
+}
+.has-warning .help-block,
+.has-warning .control-label,
+.has-warning .radio,
+.has-warning .checkbox,
+.has-warning .radio-inline,
+.has-warning .checkbox-inline,
+.has-warning.radio label,
+.has-warning.checkbox label,
+.has-warning.radio-inline label,
+.has-warning.checkbox-inline label {
+  color: #8a6d3b;
+}
+.has-warning .form-control {
+  border-color: #8a6d3b;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-warning .form-control:focus {
+  border-color: #66512c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
+}
+.has-warning .input-group-addon {
+  color: #8a6d3b;
+  border-color: #8a6d3b;
+  background-color: #fcf8e3;
+}
+.has-warning .form-control-feedback {
+  color: #8a6d3b;
+}
+.has-error .help-block,
+.has-error .control-label,
+.has-error .radio,
+.has-error .checkbox,
+.has-error .radio-inline,
+.has-error .checkbox-inline,
+.has-error.radio label,
+.has-error.checkbox label,
+.has-error.radio-inline label,
+.has-error.checkbox-inline label {
+  color: #a94442;
+}
+.has-error .form-control {
+  border-color: #a94442;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-error .form-control:focus {
+  border-color: #843534;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
+}
+.has-error .input-group-addon {
+  color: #a94442;
+  border-color: #a94442;
+  background-color: #f2dede;
+}
+.has-error .form-control-feedback {
+  color: #a94442;
+}
+.has-feedback label ~ .form-control-feedback {
+  top: 25px;
+}
+.has-feedback label.sr-only ~ .form-control-feedback {
+  top: 0;
+}
+.help-block {
+  display: block;
+  margin-top: 5px;
+  margin-bottom: 10px;
+  color: #737373;
+}
+@media (min-width: 768px) {
+  .form-inline .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .form-inline .form-control-static {
+    display: inline-block;
+  }
+  .form-inline .input-group {
+    display: inline-table;
+    vertical-align: middle;
+  }
+  .form-inline .input-group .input-group-addon,
+  .form-inline .input-group .input-group-btn,
+  .form-inline .input-group .form-control {
+    width: auto;
+  }
+  .form-inline .input-group > .form-control {
+    width: 100%;
+  }
+  .form-inline .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio,
+  .form-inline .checkbox {
+    display: inline-block;
+    margin-top: 0;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio label,
+  .form-inline .checkbox label {
+    padding-left: 0;
+  }
+  .form-inline .radio input[type="radio"],
+  .form-inline .checkbox input[type="checkbox"] {
+    position: relative;
+    margin-left: 0;
+  }
+  .form-inline .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox,
+.form-horizontal .radio-inline,
+.form-horizontal .checkbox-inline {
+  margin-top: 0;
+  margin-bottom: 0;
+  padding-top: 7px;
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox {
+  min-height: 27px;
+}
+.form-horizontal .form-group {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .control-label {
+    text-align: right;
+    margin-bottom: 0;
+    padding-top: 7px;
+  }
+}
+.form-horizontal .has-feedback .form-control-feedback {
+  right: 15px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .form-group-lg .control-label {
+    padding-top: 14.333333px;
+    font-size: 18px;
+  }
+}
+@media (min-width: 768px) {
+  .form-horizontal .form-group-sm .control-label {
+    padding-top: 6px;
+    font-size: 12px;
+  }
+}
+.btn {
+  display: inline-block;
+  margin-bottom: 0;
+  font-weight: normal;
+  text-align: center;
+  vertical-align: middle;
+  touch-action: manipulation;
+  cursor: pointer;
+  background-image: none;
+  border: 1px solid

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf b/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf
new file mode 100644
index 0000000..f7936cc
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/fonts/FontAwesome.otf differ


[15/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot
deleted file mode 100644
index 33b2bb8..0000000
Binary files a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
deleted file mode 100644
index 1ee89d4..0000000
--- a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
+++ /dev/null
@@ -1,565 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1">
-<metadata></metadata>
-<defs>
-<font id="fontawesomeregular" horiz-adv-x="1536" >
-<font-face units-per-em="1792" ascent="1536" descent="-256" />
-<missing-glyph horiz-adv-x="448" />
-<glyph unicode=" "  horiz-adv-x="448" />
-<glyph unicode="&#x09;" horiz-adv-x="448" />
-<glyph unicode="&#xa0;" horiz-adv-x="448" />
-<glyph unicode="&#xa8;" horiz-adv-x="1792" />
-<glyph unicode="&#xa9;" horiz-adv-x="1792" />
-<glyph unicode="&#xae;" horiz-adv-x="1792" />
-<glyph unicode="&#xb4;" horiz-adv-x="1792" />
-<glyph unicode="&#xc6;" horiz-adv-x="1792" />
-<glyph unicode="&#xd8;" horiz-adv-x="1792" />
-<glyph unicode="&#x2000;" horiz-adv-x="768" />
-<glyph unicode="&#x2001;" horiz-adv-x="1537" />
-<glyph unicode="&#x2002;" horiz-adv-x="768" />
-<glyph unicode="&#x2003;" horiz-adv-x="1537" />
-<glyph unicode="&#x2004;" horiz-adv-x="512" />
-<glyph unicode="&#x2005;" horiz-adv-x="384" />
-<glyph unicode="&#x2006;" horiz-adv-x="256" />
-<glyph unicode="&#x2007;" horiz-adv-x="256" />
-<glyph unicode="&#x2008;" horiz-adv-x="192" />
-<glyph unicode="&#x2009;" horiz-adv-x="307" />
-<glyph unicode="&#x200a;" horiz-adv-x="85" />
-<glyph unicode="&#x202f;" horiz-adv-x="307" />
-<glyph unicode="&#x205f;" horiz-adv-x="384" />
-<glyph unicode="&#x2122;" horiz-adv-x="1792" />
-<glyph unicode="&#x221e;" horiz-adv-x="1792" />
-<glyph unicode="&#x2260;" horiz-adv-x="1792" />
-<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
-<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
-<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
-<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
-<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
-<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
-<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
-<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
-<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
-<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
-<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
-<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
-<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
-<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
-<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
-<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
-<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
-<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
-<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
-<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
-<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
-<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
-<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
-<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
-<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
-<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
-<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
-<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
-<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
-<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
-<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
-<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
-<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
-<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
-<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
-<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
-<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
-<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
-<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
-<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
-<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
-<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
-<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
-<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
-<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
-<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
-<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
-<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
-<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
-<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
-<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
-<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
-<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
-<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
-<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
-<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
-<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
-<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
-<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
-<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
-<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
-<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
-<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf082;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 q0 119 84.5 203.5t203.5 84.5h960z" />
-<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
-<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
-<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
-<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
-<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
-<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
-<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
-<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
-<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
-<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
-<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
-<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
-<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
-<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
-<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
-<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
-<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
-<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
-<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
-<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
-<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
-<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
-<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11.5 -8.5t-9 -6.5t

<TRUNCATED>

[20/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/js/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/index.js b/flink-runtime-web/src/main/resources/web/js/index.js
new file mode 100644
index 0000000..d370765
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/js/index.js
@@ -0,0 +1,1420 @@
+angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
+  $rootScope.sidebarVisible = false;
+  return $rootScope.showSidebar = function() {
+    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
+    return $rootScope.sidebarClass = 'force-show';
+  };
+}]).value('flinkConfig', {
+  "refresh-interval": 10000
+}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
+  MainService.loadConfig().then(function(config) {
+    angular.extend(flinkConfig, config);
+    JobsService.listJobs();
+    return $interval(function() {
+      return JobsService.listJobs();
+    }, flinkConfig["refresh-interval"]);
+  });
+  Highcharts.setOptions({
+    global: {
+      useUTC: false
+    }
+  });
+  Highcharts.createElement('link', {
+    href: '//fonts.googleapis.com/css?family=Dosis:400,600',
+    rel: 'stylesheet',
+    type: 'text/css'
+  }, null, document.getElementsByTagName('head')[0]);
+  Highcharts.theme = {
+    colors: ["#7cb5ec", "#f7a35c", "#90ee7e", "#7798BF", "#aaeeee", "#ff0066", "#eeaaee", "#55BF3B", "#DF5353", "#7798BF", "#aaeeee"],
+    chart: {
+      backgroundColor: null,
+      style: {
+        fontFamily: "Dosis, sans-serif"
+      }
+    },
+    title: {
+      style: {
+        fontSize: '16px',
+        fontWeight: 'bold',
+        textTransform: 'uppercase'
+      }
+    },
+    tooltip: {
+      borderWidth: 0,
+      backgroundColor: 'rgba(219,219,216,0.8)',
+      shadow: false
+    },
+    legend: {
+      itemStyle: {
+        fontWeight: 'bold',
+        fontSize: '13px'
+      }
+    },
+    xAxis: {
+      gridLineWidth: 1,
+      labels: {
+        style: {
+          fontSize: '12px'
+        }
+      }
+    },
+    yAxis: {
+      minorTickInterval: 'auto',
+      title: {
+        style: {
+          textTransform: 'uppercase'
+        }
+      },
+      labels: {
+        style: {
+          fontSize: '12px'
+        }
+      }
+    },
+    plotOptions: {
+      candlestick: {
+        lineColor: '#404048'
+      }
+    },
+    background2: '#F0F0EA'
+  };
+  return Highcharts.setOptions(Highcharts.theme);
+}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
+  return $uiViewScrollProvider.useAnchorScroll();
+}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
+  $stateProvider.state("overview", {
+    url: "/overview",
+    views: {
+      main: {
+        templateUrl: "partials/overview.html",
+        controller: 'OverviewController'
+      }
+    }
+  }).state("running-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: true,
+    views: {
+      main: {
+        templateUrl: "partials/jobs/job.html",
+        controller: 'SingleJobController'
+      }
+    }
+  }).state("single-job.plan", {
+    url: "",
+    abstract: true,
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.plan.html",
+        controller: 'JobPlanController'
+      }
+    }
+  }).state("single-job.plan.overview", {
+    url: "",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
+        controller: 'JobPlanOverviewController'
+      }
+    }
+  }).state("single-job.plan.accumulators", {
+    url: "/accumulators",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
+        controller: 'JobPlanAccumulatorsController'
+      }
+    }
+  }).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.statistics", {
+    url: "/statistics",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.statistics.html"
+      }
+    }
+  }).state("single-job.exceptions", {
+    url: "/exceptions",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.exceptions.html",
+        controller: 'JobExceptionsController'
+      }
+    }
+  }).state("single-job.properties", {
+    url: "/properties",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.properties.html",
+        controller: 'JobPropertiesController'
+      }
+    }
+  }).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}",
+    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.log", {
+    url: "/logfile",
+    views: {
+      details: {
+        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
+      }
+    }
+  }).state("single-manager.stdout", {
+    url: "/stdout",
+    views: {
+      details: {
+        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
+      }
+    }
+  }).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'
+      }
+    }
+  });
+  return $urlRouterProvider.otherwise("/overview");
+}]);
+
+angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
+  return {
+    transclude: true,
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: "@"
+    },
+    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'label label-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
+  return {
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: '@'
+    },
+    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('tableProperty', function() {
+  return {
+    replace: true,
+    scope: {
+      value: '='
+    },
+    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
+  };
+});
+
+angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
+  var amDurationFormatExtendedFilter;
+  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
+    if (typeof value === "undefined" || value === null) {
+      return "";
+    }
+    return moment.duration(value, format).format(durationFormat, {
+      trim: false
+    });
+  };
+  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
+  return amDurationFormatExtendedFilter;
+}]).filter("humanizeText", function() {
+  return function(text) {
+    if (text) {
+      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
+    } else {
+      return '';
+    }
+  };
+}).filter("bytes", function() {
+  return function(bytes, precision) {
+    var number, units;
+    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
+      return "-";
+    }
+    if (typeof precision === "undefined") {
+      precision = 1;
+    }
+    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
+    number = Math.floor(Math.log(bytes) / Math.log(1024));
+    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
+  };
+});
+
+angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("config").success(function(data, status, headers, config) {
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
+  return JobManagerConfigService.loadConfig().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['config'] = data;
+  });
+}]).controller('JobManagerLogsController', ["$scope", "JobManagerLogsService", function($scope, JobManagerLogsService) {
+  JobManagerLogsService.loadLogs().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['log'] = data;
+  });
+  return $scope.reloadData = function() {
+    return JobManagerLogsService.loadLogs().then(function(data) {
+      return $scope.jobmanager['log'] = data;
+    });
+  };
+}]).controller('JobManagerStdoutController', ["$scope", "JobManagerStdoutService", function($scope, JobManagerStdoutService) {
+  JobManagerStdoutService.loadStdout().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['stdout'] = data;
+  });
+  return $scope.reloadData = function() {
+    return JobManagerStdoutService.loadStdout().then(function(data) {
+      return $scope.jobmanager['stdout'] = data;
+    });
+  };
+}]);
+
+angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var config;
+  config = {};
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/config").success(function(data, status, headers, config) {
+      config = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('JobManagerLogsService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var logs;
+  logs = {};
+  this.loadLogs = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/log").success(function(data, status, headers, config) {
+      logs = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('JobManagerStdoutService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var stdout;
+  stdout = {};
+  this.loadStdout = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("jobmanager/stdout").success(function(data, status, headers, config) {
+      stdout = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('running');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
+  var refresher;
+  console.log('SingleJobController');
+  $scope.jobid = $stateParams.jobid;
+  $scope.job = null;
+  $scope.plan = null;
+  $scope.vertices = null;
+  JobsService.loadJob($stateParams.jobid).then(function(data) {
+    $scope.job = data;
+    $scope.plan = data.plan;
+    return $scope.vertices = data.vertices;
+  });
+  refresher = $interval(function() {
+    return JobsService.loadJob($stateParams.jobid).then(function(data) {
+      $scope.job = data;
+      return $scope.$broadcast('reload');
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    $scope.job = null;
+    $scope.plan = null;
+    $scope.vertices = null;
+    return $interval.cancel(refresher);
+  });
+}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobPlanController');
+  $scope.nodeid = null;
+  $scope.nodeUnfolded = false;
+  $scope.stateList = JobsService.stateList();
+  $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      $scope.accumulators = null;
+      return $scope.$broadcast('reload');
+    } else {
+      $scope.nodeid = null;
+      $scope.nodeUnfolded = false;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      return $scope.accumulators = null;
+    }
+  };
+  $scope.deactivateNode = function() {
+    $scope.nodeid = null;
+    $scope.nodeUnfolded = false;
+    $scope.vertex = null;
+    $scope.subtasks = null;
+    return $scope.accumulators = null;
+  };
+  return $scope.toggleFold = function() {
+    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
+  };
+}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanOverviewController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
+    JobsService.getSubtasks($scope.nodeid).then(function(data) {
+      return $scope.subtasks = data;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanOverviewController');
+    if ($scope.nodeid) {
+      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
+        return $scope.subtasks = data;
+      });
+    }
+  });
+}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanAccumulatorsController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
+    JobsService.getAccumulators($scope.nodeid).then(function(data) {
+      $scope.accumulators = data.main;
+      return $scope.subtaskAccumulators = data.subtasks;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanAccumulatorsController');
+    if ($scope.nodeid) {
+      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
+        $scope.accumulators = data.main;
+        return $scope.subtaskAccumulators = data.subtasks;
+      });
+    }
+  });
+}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobTimelineVertexController');
+  JobsService.getVertex($stateParams.vertexId).then(function(data) {
+    return $scope.vertex = data;
+  });
+  return $scope.$on('reload', function(event) {
+    console.log('JobTimelineVertexController');
+    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
+      return $scope.vertex = data;
+    });
+  });
+}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  return JobsService.loadExceptions().then(function(data) {
+    return $scope.exceptions = data;
+  });
+}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPropertiesController');
+  return $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      return JobsService.getNode(nodeid).then(function(data) {
+        return $scope.node = data;
+      });
+    } else {
+      $scope.nodeid = null;
+      return $scope.node = null;
+    }
+  };
+}]);
+
+angular.module('flinkApp').directive('vertex', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
+    scope: {
+      data: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data.subtasks, function(subtask, i) {
+          var times;
+          times = [
+            {
+              label: "Scheduled",
+              color: "#666",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["SCHEDULED"],
+              ending_time: subtask.timestamps["DEPLOYING"],
+              type: 'regular'
+            }, {
+              label: "Deploying",
+              color: "#aaa",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["DEPLOYING"],
+              ending_time: subtask.timestamps["RUNNING"],
+              type: 'regular'
+            }
+          ];
+          if (subtask.timestamps["FINISHED"] > 0) {
+            times.push({
+              label: "Running",
+              color: "#ddd",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["RUNNING"],
+              ending_time: subtask.timestamps["FINISHED"],
+              type: 'regular'
+            });
+          }
+          return testData.push({
+            label: "(" + subtask.subtask + ") " + subtask.host,
+            times: times
+          });
+        });
+        chart = d3.timeline().stack().tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("single").labelFormat(function(label) {
+          return label;
+        }).margin({
+          left: 100,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).relativeTime();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      analyzeTime(scope.data);
+    }
+  };
+}]).directive('timeline', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline' width='0' height='0'></svg>",
+    scope: {
+      vertices: "=",
+      jobid: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl, translateLabel;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      translateLabel = function(label) {
+        return label.replace("&gt;", ">");
+      };
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data, function(vertex) {
+          if (vertex['start-time'] > -1) {
+            if (vertex.type === 'scheduled') {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#cccccc",
+                    borderColor: "#555555",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    type: vertex.type
+                  }
+                ]
+              });
+            } else {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#d9f1f7",
+                    borderColor: "#62cdea",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    link: vertex.id,
+                    type: vertex.type
+                  }
+                ]
+              });
+            }
+          }
+        });
+        chart = d3.timeline().stack().click(function(d, i, datum) {
+          if (d.link) {
+            return $state.go("single-job.timeline.vertex", {
+              jobid: scope.jobid,
+              vertexId: d.link
+            });
+          }
+        }).tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("main").margin({
+          left: 0,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).showBorderLine().showHourTimeline();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      scope.$watch(attrs.vertices, function(data) {
+        if (data) {
+          return analyzeTime(data);
+        }
+      });
+    }
+  };
+}]).directive('jobPlan', ["$timeout", function($timeout) {
+  return {
+    template: "<svg class='graph' width='500' height='400'><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: '=',
+      setNode: '&'
+    },
+    link: function(scope, elem, attrs) {
+      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
+      g = null;
+      mainZoom = d3.behavior.zoom();
+      subgraphs = [];
+      jobid = attrs.jobid;
+      mainSvgElement = elem.children()[0];
+      mainG = elem.children().children()[0];
+      mainTmpElement = elem.children()[1];
+      d3mainSvg = d3.select(mainSvgElement);
+      d3mainSvgG = d3.select(mainG);
+      d3tmpSvg = d3.select(mainTmpElement);
+      containerW = elem.width();
+      angular.element(elem.children()[0]).width(containerW);
+      scope.zoomIn = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() < 2.99) {
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          mainZoom.scale(mainZoom.scale() + 0.1);
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      scope.zoomOut = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() > 0.31) {
+          mainZoom.scale(mainZoom.scale() - 0.1);
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      createLabelEdge = function(el) {
+        var labelValue;
+        labelValue = "";
+        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
+          labelValue += "<div class='edge-label'>";
+          if (el.ship_strategy != null) {
+            labelValue += el.ship_strategy;
+          }
+          if (el.temp_mode !== undefined) {
+            labelValue += " (" + el.temp_mode + ")";
+          }
+          if (el.local_strategy !== undefined) {
+            labelValue += ",<br>" + el.local_strategy;
+          }
+          labelValue += "</div>";
+        }
+        return labelValue;
+      };
+      isSpecialIterationNode = function(info) {
+        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
+      };
+      getNodeType = function(el, info) {
+        if (info === "mirror") {
+          return 'node-mirror';
+        } else if (isSpecialIterationNode(info)) {
+          return 'node-iteration';
+        } else {
+          return 'node-normal';
+        }
+      };
+      createLabelNode = function(el, info, maxW, maxH) {
+        var labelValue, stepName;
+        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
+        if (info === "mirror") {
+          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
+        } else {
+          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
+        }
+        if (el.description === "") {
+          labelValue += "";
+        } else {
+          stepName = el.description;
+          stepName = shortenString(stepName);
+          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
+        }
+        if (el.step_function != null) {
+          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
+        } else {
+          if (isSpecialIterationNode(info)) {
+            labelValue += "<h5>" + info + " Node</h5>";
+          }
+          if (el.parallelism !== "") {
+            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
+          }
+          if (el.operator !== undefined) {
+            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
+          }
+        }
+        labelValue += "</div>";
+        return labelValue;
+      };
+      extendLabelNodeForIteration = function(id, maxW, maxH) {
+        var labelValue, svgID;
+        svgID = "svg-" + id;
+        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
+        return labelValue;
+      };
+      shortenString = function(s) {
+        var sbr;
+        if (s.charAt(0) === "<") {
+          s = s.replace("<", "&lt;");
+          s = s.replace(">", "&gt;");
+        }
+        sbr = "";
+        while (s.length > 30) {
+          sbr = sbr + s.substring(0, 30) + "<br>";
+          s = s.substring(30, s.length);
+        }
+        sbr = sbr + s;
+        return sbr;
+      };
+      createNode = function(g, data, el, isParent, maxW, maxH) {
+        if (isParent == null) {
+          isParent = false;
+        }
+        if (el.id === data.partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "partialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "partialSolution")
+          });
+        } else if (el.id === data.next_partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextPartialSolution")
+          });
+        } else if (el.id === data.workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "workset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "workset")
+          });
+        } else if (el.id === data.next_workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextWorkset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextWorkset")
+          });
+        } else if (el.id === data.solution_set) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionSet", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionSet")
+          });
+        } else if (el.id === data.solution_delta) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionDelta", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionDelta")
+          });
+        } else {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "")
+          });
+        }
+      };
+      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
+        var missingNode;
+        if (existingNodes.indexOf(pred.id) !== -1) {
+          return g.setEdge(pred.id, el.id, {
+            label: createLabelEdge(pred),
+            labelType: 'html',
+            arrowhead: 'normal'
+          });
+        } else {
+          missingNode = searchForNode(data, pred.id);
+          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
+            missingNodes.push(missingNode.id);
+            g.setNode(missingNode.id, {
+              label: createLabelNode(missingNode, "mirror"),
+              labelType: 'html',
+              "class": getNodeType(missingNode, 'mirror')
+            });
+            return g.setEdge(missingNode.id, el.id, {
+              label: createLabelEdge(missingNode),
+              labelType: 'html'
+            });
+          }
+        }
+      };
+      loadJsonToDagre = function(g, data) {
+        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
+        existingNodes = [];
+        missingNodes = [];
+        if (data.nodes != null) {
+          toIterate = data.nodes;
+        } else {
+          toIterate = data.step_function;
+          isParent = true;
+        }
+        for (k = 0, len = toIterate.length; k < len; k++) {
+          el = toIterate[k];
+          maxW = 0;
+          maxH = 0;
+          if (el.step_function) {
+            sg = new dagreD3.graphlib.Graph({
+              multigraph: true,
+              compound: true
+            }).setGraph({
+              nodesep: 20,
+              edgesep: 0,
+              ranksep: 20,
+              rankdir: "LR",
+              marginx: 10,
+              marginy: 10
+            });
+            subgraphs[el.id] = sg;
+            loadJsonToDagre(sg, el);
+            r = new dagreD3.render();
+            d3tmpSvg.select('g').call(r, sg);
+            maxW = sg.graph().width;
+            maxH = sg.graph().height;
+            angular.element(mainTmpElement).empty();
+          }
+          createNode(g, data, el, isParent, maxW, maxH);
+          existingNodes.push(el.id);
+          if (el.inputs != null) {
+            ref = el.inputs;
+            for (l = 0, len1 = ref.length; l < len1; l++) {
+              pred = ref[l];
+              createEdge(g, data, el, existingNodes, pred, missingNodes);
+            }
+          }
+        }
+        return g;
+      };
+      searchForNode = function(data, nodeID) {
+        var el, i, j;
+        for (i in data.nodes) {
+          el = data.nodes[i];
+          if (el.id === nodeID) {
+            return el;
+          }
+          if (el.step_function != null) {
+            for (j in el.step_function) {
+              if (el.step_function[j].id === nodeID) {
+                return el.step_function[j];
+              }
+            }
+          }
+        }
+      };
+      drawGraph = function(data) {
+        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
+        g = new dagreD3.graphlib.Graph({
+          multigraph: true,
+          compound: true
+        }).setGraph({
+          nodesep: 70,
+          edgesep: 0,
+          ranksep: 50,
+          rankdir: "LR",
+          marginx: 40,
+          marginy: 40
+        });
+        loadJsonToDagre(g, data);
+        renderer = new dagreD3.render();
+        d3mainSvgG.call(renderer, g);
+        for (i in subgraphs) {
+          sg = subgraphs[i];
+          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
+        }
+        newScale = 0.5;
+        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
+        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
+        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
+        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
+        mainZoom.on("zoom", function() {
+          var ev;
+          ev = d3.event;
+          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
+        });
+        mainZoom(d3mainSvg);
+        return d3mainSvgG.selectAll('.node').on('click', function(d) {
+          return scope.setNode({
+            nodeid: d
+          });
+        });
+      };
+      scope.$watch(attrs.plan, function(newPlan) {
+        if (newPlan) {
+          return drawGraph(newPlan);
+        }
+      });
+    }
+  };
+}]);
+
+angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
+  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
+  currentJob = null;
+  currentPlan = null;
+  deferreds = {};
+  jobs = {
+    running: [],
+    finished: [],
+    cancelled: [],
+    failed: []
+  };
+  jobObservers = [];
+  notifyObservers = function() {
+    return angular.forEach(jobObservers, function(callback) {
+      return callback();
+    });
+  };
+  this.registerObserver = function(callback) {
+    return jobObservers.push(callback);
+  };
+  this.unRegisterObserver = function(callback) {
+    var index;
+    index = jobObservers.indexOf(callback);
+    return jobObservers.splice(index, 1);
+  };
+  this.stateList = function() {
+    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
+  };
+  this.translateLabelState = function(state) {
+    switch (state.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(list) {
+    return angular.forEach(list, function(item, jobKey) {
+      if (!(item['end-time'] > -1)) {
+        return item['end-time'] = item['start-time'] + item['duration'];
+      }
+    });
+  };
+  this.processVertices = function(data) {
+    angular.forEach(data.vertices, function(vertex, i) {
+      return vertex.type = 'regular';
+    });
+    return data.vertices.unshift({
+      name: 'Scheduled',
+      'start-time': data.timestamps['CREATED'],
+      'end-time': data.timestamps['CREATED'] + 1,
+      type: 'scheduled'
+    });
+  };
+  this.listJobs = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("joboverview").success((function(_this) {
+      return function(data, status, headers, config) {
+        angular.forEach(data, function(list, listKey) {
+          switch (listKey) {
+            case 'running':
+              return jobs.running = _this.setEndTimes(list);
+            case 'finished':
+              return jobs.finished = _this.setEndTimes(list);
+            case 'cancelled':
+              return jobs.cancelled = _this.setEndTimes(list);
+            case 'failed':
+              return jobs.failed = _this.setEndTimes(list);
+          }
+        });
+        deferred.resolve(jobs);
+        return notifyObservers();
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getJobs = function(type) {
+    return jobs[type];
+  };
+  this.getAllJobs = function() {
+    return jobs;
+  };
+  this.loadJob = function(jobid) {
+    currentJob = null;
+    deferreds.job = $q.defer();
+    $http.get("jobs/" + jobid).success((function(_this) {
+      return function(data, status, headers, config) {
+        _this.setEndTimes(data.vertices);
+        _this.processVertices(data);
+        return $http.get("jobs/" + jobid + "/config").success(function(jobConfig) {
+          data = angular.extend(data, jobConfig);
+          currentJob = data;
+          return deferreds.job.resolve(currentJob);
+        });
+      };
+    })(this));
+    return deferreds.job.promise;
+  };
+  this.getNode = function(nodeid) {
+    var deferred, seekNode;
+    seekNode = function(nodeid, data) {
+      var j, len, node, sub;
+      for (j = 0, len = data.length; j < len; j++) {
+        node = data[j];
+        if (node.id === nodeid) {
+          return node;
+        }
+        if (node.step_function) {
+          sub = seekNode(nodeid, node.step_function);
+        }
+        if (sub) {
+          return sub;
+        }
+      }
+      return null;
+    };
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var foundNode;
+        foundNode = seekNode(nodeid, currentJob.plan.nodes);
+        foundNode.vertex = _this.seekVertex(nodeid);
+        return deferred.resolve(foundNode);
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.seekVertex = function(nodeid) {
+    var j, len, ref, vertex;
+    ref = currentJob.vertices;
+    for (j = 0, len = ref.length; j < len; j++) {
+      vertex = ref[j];
+      if (vertex.id === nodeid) {
+        return vertex;
+      }
+    }
+    return null;
+  };
+  this.getVertex = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var vertex;
+        vertex = _this.seekVertex(vertexid);
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
+          vertex.subtasks = data.subtasks;
+          return deferred.resolve(vertex);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getSubtasks = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
+          var subtasks;
+          subtasks = data.subtasks;
+          return deferred.resolve(subtasks);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getAccumulators = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
+          var accumulators;
+          accumulators = data['user-accumulators'];
+          return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
+            var subtaskAccumulators;
+            subtaskAccumulators = data.subtasks;
+            return deferred.resolve({
+              main: accumulators,
+              subtasks: subtaskAccumulators
+            });
+          });
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.loadExceptions = function() {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
+          currentJob.exceptions = exceptions;
+          return deferred.resolve(exceptions);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
+  var refresh;
+  $scope.jobObserver = function() {
+    $scope.runningJobs = JobsService.getJobs('running');
+    return $scope.finishedJobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  $scope.jobObserver();
+  OverviewService.loadOverview().then(function(data) {
+    return $scope.overview = data;
+  });
+  refresh = $interval(function() {
+    return OverviewService.loadOverview().then(function(data) {
+      return $scope.overview = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]);
+
+angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var overview;
+  overview = {};
+  this.loadOverview = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("overview").success(function(data, status, headers, config) {
+      overview = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('AllTaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
+  var refresh;
+  TaskManagersService.loadManagers().then(function(data) {
+    return $scope.managers = data;
+  });
+  refresh = $interval(function() {
+    return TaskManagersService.loadManagers().then(function(data) {
+      return $scope.managers = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]).controller('SingleTaskManagerController', ["$scope", "$stateParams", "SingleTaskManagerService", "$interval", "flinkConfig", function($scope, $stateParams, SingleTaskManagerService, $interval, flinkConfig) {
+  var refresh;
+  $scope.metrics = {};
+  SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
+    return $scope.metrics = data[0];
+  });
+  refresh = $interval(function() {
+    return SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
+      return $scope.metrics = data[0];
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]);
+
+angular.module('flinkApp').directive('livechart', function() {
+  return {
+    link: function(scope, element, attrs) {
+      var getChartOptions, getChartType, getKey1, getKey2, getKey3, getKey4, getYAxisTitle, updateCharts;
+      getChartType = function() {
+        if (attrs.key === "cpuLoad") {
+          return "spline";
+        } else {
+          return "area";
+        }
+      };
+      getYAxisTitle = function() {
+        if (attrs.key === "cpuLoad") {
+          return "CPU Usage(%)";
+        } else {
+          return "Memory(MB)";
+        }
+      };
+      getKey1 = function() {
+        return "memory.total." + attrs.key;
+      };
+      getKey2 = function() {
+        return "memory.heap." + attrs.key;
+      };
+      getKey3 = function() {
+        return "memory.non-heap." + attrs.key;
+      };
+      getKey4 = function() {
+        return "cpuLoad";
+      };
+      getChartOptions = function() {
+        return {
+          title: {
+            text: ' '
+          },
+          chart: {
+            type: getChartType(),
+            zoomType: 'x'
+          },
+          xAxis: {
+            type: 'datetime'
+          },
+          yAxis: {
+            title: {
+              text: getYAxisTitle()
+            },
+            min: attrs.key === "cpuLoad" ? 0 : void 0,
+            max: attrs.key === "cpuLoad" ? 100 : void 0
+          },
+          series: [
+            {
+              name: "Memory: Total",
+              id: getKey1(),
+              data: [],
+              color: "#7cb5ec"
+            }, {
+              name: "Memory: Heap",
+              id: getKey2(),
+              data: [],
+              color: "#434348"
+            }, {
+              name: "Memory: Non-Heap",
+              id: getKey3(),
+              data: [],
+              color: "#90ed7d"
+            }, {
+              name: "CPU Usage",
+              id: getKey4(),
+              data: [],
+              color: "#f7a35c",
+              showInLegend: false
+            }
+          ],
+          legend: {
+            enabled: false
+          },
+          tooltip: {
+            shared: true
+          },
+          exporting: {
+            enabled: false
+          },
+          credits: {
+            enabled: false
+          }
+        };
+      };
+      if (element.highcharts() == null) {
+        element.highcharts(getChartOptions());
+      }
+      scope.$watch(attrs.data, function(value) {
+        return updateCharts(value);
+      });
+      return updateCharts = function(value) {
+        return (function(value) {
+          var chart, divider, heartbeat;
+          heartbeat = value.timeSinceLastHeartbeat;
+          chart = element.highcharts();
+          if (attrs.key === "cpuLoad") {
+            return chart.get(getKey4()).addPoint([heartbeat, +((value.metrics.gauges[getKey4()].value * 100).toFixed(2))], true, false);
+          } else {
+            divider = 1048576;
+            chart.get(getKey1()).addPoint([heartbeat, +((value.metrics.gauges[getKey1()].value / divider).toFixed(2))], true, false);
+            chart.get(getKey2()).addPoint([heartbeat, +((value.metrics.gauges[getKey2()].value / divider).toFixed(2))], true, false);
+            return chart.get(getKey3()).addPoint([heartbeat, +((value.metrics.gauges[getKey3()].value / divider).toFixed(2))], true, false);
+          }
+        })(value);
+      };
+    }
+  };
+});
+
+angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadManagers = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("taskmanagers").success(function(data, status, headers, config) {
+      return deferred.resolve(data['taskmanagers']);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]).service('SingleTaskManagerService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadMetrics = function(taskmanagerid) {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("taskmanagers/" + taskmanagerid).success(function(data, status, headers, config) {
+      return deferred.resolve(data['taskmanagers']);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5
 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQUNILFlBQVksYUFBYSxLQUFLLFNBQUMsUUFBRDtJQUM1QixRQUFRLE9BQU8sYUFBYTtJQUU1QixZQUFZO0lDNUJaLE9EOEJBLFVBQVUsV0FBQTtNQzdCUixPRDhCQSxZQUFZO09BQ1osWUFBWTs7RUFFaEIsV0FBVyxXQUFXO0lBQ3BCLFFBQVE7TUFDTixRQUFROzs7RUFZWixXQUFXLGNBQWMsUUFBUTtJQUNoQyxNQUFNO0lBQ04sS0FBSztJQUNMLE1BQU07S0FDSixNQUFNLFNBQVMscUJBQXFCLFFBQVE7RUFFL0MsV0FBVyxRQUFRO0lBQ2xCLFFBQVEsQ0FBQyxXQUFXLFdBQVcsV0FB
 VyxXQUFXLFdBQVcsV0FBVyxXQUMxRSxXQUFXLFdBQVcsV0FBVztJQUNsQyxPQUFPO01BQ04saUJBQWlCO01BQ2pCLE9BQU87UUFDTixZQUFZOzs7SUFHZCxPQUFPO01BQ04sT0FBTztRQUNOLFVBQVU7UUFDVixZQUFZO1FBQ1osZUFBZTs7O0lBR2pCLFNBQVM7TUFDUixhQUFhO01BQ2IsaUJBQWlCO01BQ2pCLFFBQVE7O0lBRVQsUUFBUTtNQUNQLFdBQVc7UUFDVixZQUFZO1FBQ1osVUFBVTs7O0lBR1osT0FBTztNQUNOLGVBQWU7TUFDZixRQUFRO1FBQ1AsT0FBTztVQUNOLFVBQVU7Ozs7SUFJYixPQUFPO01BQ04sbUJBQW1CO01BQ25CLE9BQU87UUFDTixPQUFPO1VBQ04sZUFBZTs7O01BR2pCLFFBQVE7UUFDUCxPQUFPO1VBQ04sVUFBVTs7OztJQUliLGFBQWE7TUFDWixhQUFhO1FBQ1osV0FBVzs7O0lBSWIsYUFBYTs7RUN4Q2QsT0Q0Q0EsV0FBVyxXQUFXLFdBQVc7SUFLbEMsaUNBQU8sU0FBQyx1QkFBRDtFQy9DTixPRGdEQSxzQkFBc0I7SUFJdkIsZ0RBQU8sU0FBQyxnQkFBZ0Isb0JBQWpCO0VBQ04sZUFBZSxNQUFNLFlBQ25CO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGNBQ0w7SUFBQSxLQUFLO0lBQ
 0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLG1CQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSw0QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0NBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHVCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sOEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFFBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0g7SUFBQSxLQUFLO0
 lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRW5CLE1BQU0sMEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sY0FDSDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7OztLQUVwQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7Ozs7RUMxQmxCLE9ENEJBLG1CQUFtQixVQUFVOztBQzFCL0I7QUNoUEEsUUFBUSxPQUFPLFlBSWQsVUFBVSwyQkFBVyxTQUFDLGFBQUQ7RUNyQnBCLE9Ec0JBO0lBQUEsWUFBWTtJQUNaLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsaUJBQWlCLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJNUQsVUFBVSxvQ0F
 Bb0IsU0FBQyxhQUFEO0VDckI3QixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsc0NBQXNDLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJakYsVUFBVSxpQkFBaUIsV0FBQTtFQ3JCMUIsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLE9BQU87O0lBRVQsVUFBVTs7O0FDbEJaO0FDcEJBLFFBQVEsT0FBTyxZQUVkLE9BQU8sb0RBQTRCLFNBQUMscUJBQUQ7RUFDbEMsSUFBQTtFQUFBLGlDQUFpQyxTQUFDLE9BQU8sUUFBUSxnQkFBaEI7SUFDL0IsSUFBYyxPQUFPLFVBQVMsZUFBZSxVQUFTLE1BQXREO01BQUEsT0FBTzs7SUNoQlAsT0RrQkEsT0FBTyxTQUFTLE9BQU8sUUFBUSxPQUFPLGdCQUFnQjtNQUFFLE1BQU07OztFQUVoRSwrQkFBK0IsWUFBWSxvQkFBb0I7RUNmL0QsT0RpQkE7SUFFRCxPQUFPLGdCQUFnQixXQUFBO0VDakJ0QixPRGtCQSxTQUFDLE1BQUQ7SUFFRSxJQUFHLE1BQUg7TUNsQkUsT0RrQlcsS0FBSyxRQUFRLFNBQVMsS0FBSyxRQUFRLFdBQVU7V0FBMUQ7TUNoQkUsT0RnQmlFOzs7R0FFdEUsT0FBTyxTQUFTLFdBQUE7RUNkZixPRGVBLFNBQUMsT0FBTyxXQUFSO0lBQ0UsSUFBQSxRQUFBO0lBQUEsSUFBZSxNQUFNLFdBQVcsV0FBVyxDQUFJLFNBQVMsUUFBeEQ7TUFBQSxPQUFPOztJQUNQLElBQWtCLE9BQU8sY0FB
 YSxhQUF0QztNQUFBLFlBQVk7O0lBQ1osUUFBUSxDQUFFLFNBQVMsTUFBTSxNQUFNLE1BQU0sTUFBTTtJQUMzQyxTQUFTLEtBQUssTUFBTSxLQUFLLElBQUksU0FBUyxLQUFLLElBQUk7SUNUL0MsT0RVQSxDQUFDLFFBQVEsS0FBSyxJQUFJLE1BQU0sS0FBSyxNQUFNLFVBQVUsUUFBUSxhQUFhLE1BQU0sTUFBTTs7O0FDUGxGO0FDaEJBLFFBQVEsT0FBTyxZQUVkLFFBQVEsOENBQWUsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDdEIsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFVBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9Ec0JBOztBQ3BCRjtBQ09BLFFBQVEsT0FBTyxZQUVkLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQ25CeEMsT0RvQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7SUFDeEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNsQnRCLE9EbUJBLE9BQU8sV0FBVyxZQUFZOztJQUVqQyxXQUFXLGdFQUE0QixTQUFDLFFBQVEsdUJBQVQ7RUFDdEMsc0JBQXNCLFdBQVcsS0FBSyxTQUFDLE1BQUQ7SUFDcEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNqQnRCLE9Ea0JBLE9BQU8sV0FBVyxTQUFTOztFQ2hCN0IsT0RrQkEsT0FBTyxhQUFhLFdBQUE7SUNqQmxCLE9Ea0JBLHNCQUFzQixXQUFXLEtBQ
 UssU0FBQyxNQUFEO01DakJwQyxPRGtCQSxPQUFPLFdBQVcsU0FBUzs7O0lBRWhDLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQUN4Qyx3QkFBd0IsYUFBYSxLQUFLLFNBQUMsTUFBRDtJQUN4QyxJQUFJLE9BQUEsY0FBQSxNQUFKO01BQ0UsT0FBTyxhQUFhOztJQ2Z0QixPRGdCQSxPQUFPLFdBQVcsWUFBWTs7RUNkaEMsT0RnQkEsT0FBTyxhQUFhLFdBQUE7SUNmbEIsT0RnQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7TUNmeEMsT0RnQkEsT0FBTyxXQUFXLFlBQVk7Ozs7QUNacEM7QUNkQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDBEQUEyQixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNsQyxJQUFBO0VBQUEsU0FBUztFQUVULEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxxQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxTQUFTO01DcEJULE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9EcUJBO0lBRUQsUUFBUSx3REFBeUIsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDaEMsSUFBQTtFQUFBLE9BQU87RUFFUCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksa0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsT0FBTztNQ3RCUCxPRHVCQSxTQUFTLFFBQVE7O0lDckJuQixPRHVCQSxTQUFTOztFQ3JCWCxPRHVCQTtJQUVELFFBQVEsME
 RBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHFCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUN4QlQsT0R5QkEsU0FBUyxRQUFROztJQ3ZCbkIsT0R5QkEsU0FBUzs7RUN2QlgsT0R5QkE7O0FDdkJGO0FDdEJBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyxxSEFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQVksYUFBYSxXQUFyRTtFQUNqQyxJQUFBO0VBQUEsUUFBUSxJQUFJO0VBRVosT0FBTyxRQUFRLGFBQWE7RUFDNUIsT0FBTyxNQUFNO0VBQ2I
 sT0FBTyxPQUFPO0VBQ2QsT0FBTyxXQUFXO0VBRWxCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUFDM0MsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPLEtBQUs7SUMxQm5CLE9EMkJBLE9BQU8sV0FBVyxLQUFLOztFQUV6QixZQUFZLFVBQVUsV0FBQTtJQzFCcEIsT0QyQkEsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtNQUMzQyxPQUFPLE1BQU07TUMxQmIsT0Q0QkEsT0FBTyxXQUFXOztLQUVwQixZQUFZO0VDM0JkLE9ENkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUFDckIsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPO0lBQ2QsT0FBTyxXQUFXO0lDNUJsQixPRDhCQSxVQUFVLE9BQU87O0lBS3BCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDL0IsUUFBUSxJQUFJO0VBRVosT0FBTyxTQUFTO0VBQ2hCLE9BQU8sZUFBZTtFQUN0QixPQUFPLFlBQVksWUFBWTtFQUUvQixPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01BQ2hCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUFDbEIsT0FBTyxlQUFlO01DbEN0QixPRG9DQSxPQUFPLFdBQVc7V0FOcEI7TUFTRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxlQUFlO01BQ3RCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUNwQ2xCLE9EcUNBLE9BQU8sZUFBZTs7O0VBRTFCLE9BQU8saUJBQWlCLFdBQUE7SUFDdEIsT0FBTyxTQUFTO0lBQ2hCLE9BQU8s
 ZUFBZTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxXQUFXO0lDbkNsQixPRG9DQSxPQUFPLGVBQWU7O0VDbEN4QixPRG9DQSxPQUFPLGFBQWEsV0FBQTtJQ25DbEIsT0RvQ0EsT0FBTyxlQUFlLENBQUMsT0FBTzs7SUFJakMsV0FBVyx1REFBNkIsU0FBQyxRQUFRLGFBQVQ7RUFDdkMsUUFBUSxJQUFJO0VBRVosSUFBRyxPQUFPLFdBQVksQ0FBQyxPQUFPLFVBQVUsQ0FBQyxPQUFPLE9BQU8sS0FBdkQ7SUFDRSxZQUFZLFlBQVksT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01DdEMxQyxPRHVDQSxPQUFPLFdBQVc7OztFQ3BDdEIsT0RzQ0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQUNaLElBQUcsT0FBTyxRQUFWO01DckNFLE9Ec0NBLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUNyQzFDLE9Ec0NBLE9BQU8sV0FBVzs7OztJQUl6QixXQUFXLDJEQUFpQyxTQUFDLFFBQVEsYUFBVDtFQUMzQyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxlQUF2RDtJQUNFLFlBQVksZ0JBQWdCLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtNQUM5QyxPQUFPLGVBQWUsS0FBSztNQ3RDM0IsT0R1Q0EsT0FBTyxzQkFBc0IsS0FBSzs7O0VDcEN0QyxPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRL
 EtBQUssU0FBQyxNQUFEO1FBQzlDLE9BQU8sZUFBZSxLQUFLO1FDckMzQixPRHNDQSxPQUFPLHNCQUFzQixLQUFLOzs7O0lBSXpDLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDekMsUUFBUSxJQUFJO0VBRVosWUFBWSxVQUFVLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ3RDaEQsT0R1Q0EsT0FBTyxTQUFTOztFQ3JDbEIsT0R1Q0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQ3RDWixPRHVDQSxZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO01DdENoRCxPRHVDQSxPQUFPLFNBQVM7OztJQUlyQixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDdkNyQyxPRHdDQSxZQUFZLGlCQUFpQixLQUFLLFNBQUMsTUFBRDtJQ3ZDaEMsT0R3Q0EsT0FBTyxhQUFhOztJQUl2QixXQUFXLHFEQUEyQixTQUFDLFFBQVEsYUFBVDtFQUNyQyxRQUFRLElBQUk7RUN6Q1osT0QyQ0EsT0FBTyxhQUFhLFNBQUMsUUFBRDtJQUNsQixJQUFHLFdBQVUsT0FBTyxRQUFwQjtNQUNFLE9BQU8sU0FBUztNQzFDaEIsT0Q0Q0EsWUFBWSxRQUFRLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUMzQy9CLE9ENENBLE9BQU8sT0FBTzs7V0FKbEI7TUFPRSxPQUFPLFNBQVM7TUMzQ2hCLE9ENENBLE9BQU8sT0FBTzs7OztBQ3hDcEI7QUNuSEEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TU
 FBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBO01BQUEsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUztNQUVyQyxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsT0FBQSxLQUFBO1FBQUEsR0FBRyxPQUFPLE9BQU8sVUFBVSxLQUFLO1FBRWhDLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsU0FBUyxHQUFWO1VBQzdCLElBQUE7VUFBQSxRQUFRO1lBQ047Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNO2VBRVI7Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUFJVixJQUFHLFFBQVEsV0FBVyxjQUFjLEdBQXBDO1lBQ0UsTUFBTSxLQUFLO2NBQ1QsT0FBTztjQUNQLE9BQU87Y0FDUCxhQUFhO2NBQ2IsZUFBZSxRQUFRLFdBQVc7Y0FDbEMsYUFBYSxRQUFRLFdBQVc7Y0FDaEMsTUFBTTs7O1VDdEJSLE9EeUJGLFNBQVMsS0FBSztZQUNaLE9BQU8sTUFBSSxRQUFRLFVBQVEsT0FBSSxRQUFRO1lBQ3ZDLE9BQU87OztRQUdYLFFBQVEsR0FBRyxXQUFXLFFBQ3JCLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLFVBQVU7V0FFWCxPQUFPLFVBQ1AsWUFBWSxTQUFDLE9BQUQ7VUM1QlQ
 sT0Q2QkY7V0FFRCxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTtXQUM5QyxXQUFXLElBQ1g7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQ2hDckIsT0RpQ0E7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLFVBQVU7TUFDVixPQUFPOztJQUVULE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGlCQUFpQixTQUFDLE9BQUQ7UUNqQ2IsT0RrQ0YsTUFBTSxRQUFRLFFBQVE7O01BRXhCLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxNQUFNLFNBQUMsUUFBRDtVQUNwQixJQUFHLE9BQU8sZ0JBQWdCLENBQUMsR0FBM0I7WUFDRSxJQUFHLE9BQU8sU0FBUSxhQUFsQjtjQ2xDSSxPRG1DRixTQUFTLEtBQ1A7Z0JBQUEsT0FBTztrQkFDTDtvQkFBQSxPQUFPLGVBQWUsT0FBTztvQkFDN0IsT0FBTztvQkFDUCxhQUFhO29CQUNiLGVBQWUsT0FBTztvQkFDdEIsYUFBYSxPQUFPO29CQUNwQixNQUFNLE9BQU87Ozs7bUJBUm5CO2NDckJJLE9EZ0NGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29C
 QUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTztvQkFDYixNQUFNLE9BQU87Ozs7Ozs7UUFHdkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUFDbEMsSUFBRyxFQUFFLE1BQUw7WUMxQkksT0QyQkYsT0FBTyxHQUFHLDhCQUE4QjtjQUFFLE9BQU8sTUFBTTtjQUFPLFVBQVUsRUFBRTs7O1dBRzdFLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBR3ZCLFVBQVU7V0FFWCxPQUFPLFFBQ1AsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7V0FDNUMsV0FBVyxJQUNYLGlCQUNBO1FDMUJDLE9ENEJGLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUs7O01BRVIsTUFBTSxPQUFPLE1BQU0sVUFBVSxTQUFDLE1BQUQ7UUFDM0IsSUFBcUIsTUFBckI7VUM3QkksT0Q2QkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUM3QnBCLE9EOEJBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNO01BQ04sU0FBUzs7SUFFWCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsR0FBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLElBQUk7TUFDSixXQ
 UFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFLckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDMUN2QixPRDZDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM1Q3ZCLE9EK0NGLFdBQVcsS0FBSyxhQU
 FhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3RDZCxPRHVDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDeENyQixPRHlDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDeENJLE9EeUNGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUN6Q0QsT0QwQ0Y7ZUFERztVQ3ZDRCxPRDJDQTs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFFaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSx1QkFBdUIsUUFBUSxhQUFhLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHbEgsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLFdBQVc7ZUFEbkU7VUFHRSxjQUFjLDJCQUEyQixHQUFHLFdBQVc7O1FBQ3pELElBQUcsR0FBRyxnQkFBZSxJQUF
 yQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUF3RixHQUFHLGFBQVksV0FBdkc7WUFBQSxjQUFjLG9CQUFvQixjQUFjLEdBQUcscUJBQXFCOzs7UUFHMUUsY0FBYztRQzNDWixPRDRDRjs7TUFHRiw4QkFBOEIsU0FBQyxJQUFJLE1BQU0sTUFBWDtRQUM1QixJQUFBLFlBQUE7UUFBQSxRQUFRLFNBQVM7UUFFakIsYUFBYSxpQkFBaUIsUUFBUSxhQUFhLE9BQU8sYUFBYSxPQUFPO1FDNUM1RSxPRDZDRjs7TUFHRixnQkFBZ0IsU0FBQyxHQUFEO1FBRWQsSUFBQTtRQUFBLElBQUcsRUFBRSxPQUFPLE9BQU0sS0FBbEI7VUFDRSxJQUFJLEVBQUUsUUFBUSxLQUFLO1VBQ25CLElBQUksRUFBRSxRQUFRLEtBQUs7O1FBQ3JCLE1BQU07UUFDTixPQUFNLEVBQUUsU0FBUyxJQUFqQjtVQUNFLE1BQU0sTUFBTSxFQUFFLFVBQVUsR0FBRyxNQUFNO1VBQ2pDLElBQUksRUFBRSxVQUFVLElBQUksRUFBRTs7UUFDeEIsTUFBTSxNQUFNO1FDM0NWLE9ENENGOztNQUVGLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxVQUFrQixNQUFNLE1BQXRDO1FDM0NU
 LElBQUksWUFBWSxNQUFNO1VEMkNDLFdBQVc7O1FBRXBDLElBQUcsR0FBRyxPQUFNLEtBQUssa0JBQWpCO1VDekNJLE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLG1CQUFtQixNQUFNO1lBQ3BELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyx1QkFBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksdUJBQXVCLE1BQU07WUFDeEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLFNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLFdBQVcsTUFBTTtZQUM1QyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGdCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBT
 yxnQkFBZ0IsSUFBSSxpQkFBaUIsTUFBTTtZQUNsRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBSnRCO1VDbkNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLElBQUksTUFBTTtZQUNyQyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7Ozs7TUFFN0IsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTSxjQUFuQztRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3RDSSxPRHVDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFFdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxhQUFhLFFBQVEsWUFBWSxNQUFNLENBQUMsSUFBL0Q7WUFDRSxhQUFhLEtBQUssWUFBWTtZQUM5QixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ3RDaEMsT0R3Q0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsY0FBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBQ2hCLGVBQWU7UUFFZixJQUFHLE
 tBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUN2Q0ksS0FBSyxVQUFVO1VEd0NqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLFVBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDMUNJLE9BQU8sSUFBSTtjRDJDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTTs7OztRQ3RDakQsT0R3Q0Y7O01BR0YsZ0JBQWdCLFNBQUMsTUFBTSxRQUFQO1FBQ2QsSUFBQSxJQUFBLEdBQUE7UUFBQSxLQUFBLEtBQUEsS0FBQSxPQUFBO1VBQ0UsS0FBSyx
 LQUFLLE1BQU07VUFDaEIsSUFBYyxHQUFHLE9BQU0sUUFBdkI7WUFBQSxPQUFPOztVQUdQLElBQUcsR0FBQSxpQkFBQSxNQUFIO1lBQ0UsS0FBQSxLQUFBLEdBQUEsZUFBQTtjQUNFLElBQStCLEdBQUcsY0FBYyxHQUFHLE9BQU0sUUFBekQ7Z0JBQUEsT0FBTyxHQUFHLGNBQWM7Ozs7OztNQUVoQyxZQUFZLFNBQUMsTUFBRDtRQUNWLElBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQ2pDSSxLQUFLLFVBQVU7VURrQ2pCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdB
 QUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ25DTixPRG9DRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUFFckYsU0FBUztRQ25DUCxPRHFDRixXQUFXLFVBQVUsU0FBUyxHQUFHLFNBQVMsU0FBQyxHQUFEO1VDcEN0QyxPRHFDRixNQUFNLFFBQVE7WUFBRSxRQUFROzs7O01BRTVCLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDakNJLE9EaUNKLFVBQVU7Ozs7OztBQzNCaEI7QUMxYUEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBRWQsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3JCaEIsT0RzQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DckI1QixPRHNCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDcEJsQixPRHFCQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNuQjdCLE9Eb0JBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ25CWCxPRG9CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc
 0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDNUJILE9ENEJtQjtNQUR2QixLQUVPO1FDM0JILE9EMkJpQjtNQUZyQixLQUdPO1FDMUJILE9EMEJvQjtNQUh4QixLQUlPO1FDekJILE9EeUJvQjtNQUp4QixLQUtPO1FDeEJILE9Ed0JrQjtNQUx0QixLQU1PO1FDdkJILE9EdUJvQjtNQU54QixLQU9PO1FDdEJILE9Ec0JrQjtNQVB0QixLQVFPO1FDckJILE9EcUJnQjtNQVJwQjtRQ1hJLE9Eb0JHOzs7RUFFVCxLQUFDLGNBQWMsU0FBQyxNQUFEO0lDbEJiLE9EbUJBLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxRQUFQO01BQ3BCLElBQUEsRUFBTyxLQUFLLGNBQWMsQ0FBQyxJQUEzQjtRQ2xCRSxPRG1CQSxLQUFLLGNBQWMsS0FBSyxnQkFBZ0IsS0FBSzs7OztFQUVuRCxLQUFDLGtCQUFrQixTQUFDLE1BQUQ7SUFDakIsUUFBUSxRQUFRLEtBQUssVUFBVSxTQUFDLFFBQVEsR0FBVDtNQ2hCN0IsT0RpQkEsT0FBTyxPQUFPOztJQ2ZoQixPRGlCQSxLQUFLLFNBQVMsUUFBUTtNQUNwQixNQUFNO01BQ04sY0FBYyxLQUFLLFdBQVc7TUFDOUIsWUFBWSxLQUFLLFdBQVcsYUFBYTtNQUN6QyxNQUFNOzs7RUFHVixLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZUFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1
 lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxVQUFVLE9BQ25CLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFVBQVUsUUFBUSxXQUMzQixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE
 7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQyxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQzlELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RF
 RixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsVUFDbkQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDOUQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDOUQsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ES
 nlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUNwQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFlBQ1
 QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLDJGQUE2QixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDdkMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDakJkLE9EbUJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNsQnJCLE9EbUJBLFVBQVUsT0FBTzs7SUFFcEIsV0FBVyxrSEFBK0IsU0FBQyxRQUFRLGNBQWMsMEJBQTBCLFdBQVcsYUFBNUQ7RUFDekMsSUFBQTtFQUFBLE9BQU8sVUFBVTtFQUNqQix5QkFBeUIsWUFBWSxhQUFhLGVBQWUsS0FBSyxTQUFDLE1BQUQ7SUNqQnBFLE9Ea0JFLE9BQU8sVUFBVSxLQUFLOztFQUV4QixVQUFVLFVBQVUsV0FBQTtJQ2pCcEIsT0RrQkUseUJBQXlCLFlBQVksYUFBYSxlQUFlLEtBQUssU0FBQyxNQUFEO01DakJ0RSxPRGtCRSxPQUFPLFVBQVUsS0FBSzs7S0FDeEIsWUFBWTtFQ2hCaEIsT0RrQkUsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2pCdkIsT0RrQkUsVUFBVSxPQUFPOzs7QUNmdkI7QUNWQSxRQUFRLE9BQU8sWUFFZCxVQUFVLGFBQWEsV0FBQTtFQ25CdEI
 sT0RvQkE7SUFDRSxNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01BQ0osSUFBQSxpQkFBQSxjQUFBLFNBQUEsU0FBQSxTQUFBLFNBQUEsZUFBQTtNQUFBLGVBQWUsV0FBQTtRQUNiLElBQUcsTUFBTSxRQUFPLFdBQWhCO1VDbEJFLE9EbUJBO2VBREY7VUNoQkUsT0RtQkE7OztNQUVKLGdCQUFnQixXQUFBO1FBQ2QsSUFBRyxNQUFNLFFBQU8sV0FBaEI7VUNqQkUsT0RrQkE7ZUFERjtVQ2ZFLE9Ea0JBOzs7TUFFSixVQUFVLFdBQUE7UUNoQlIsT0RpQkEsa0JBQWtCLE1BQU07O01BQzFCLFVBQVUsV0FBQTtRQ2ZSLE9EZ0JBLGlCQUFpQixNQUFNOztNQUN6QixVQUFVLFdBQUE7UUNkUixPRGVBLHFCQUFxQixNQUFNOztNQUM3QixVQUFVLFdBQUE7UUNiUixPRGNBOztNQUVGLGtCQUFrQixXQUFBO1FDYmhCLE9EYXNCO1VBQ3RCLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFBQyxNQUFNO1lBQWdCLFVBQVU7O1VBQ3hDLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFDTCxPQUFPO2NBQUMsTUFBTTs7WUFDZCxLQUFVLE1BQU0sUUFBTyxZQUFsQixJQUFBLEtBQUE7WUFDTCxLQUFZLE1BQU0sUUFBTyxZQUFwQixNQUFBLEtBQUE7O1VBRVAsUUFBUTtZQUNOO2NBQUMsTUFBTTtjQUFpQixJQUFJO2NBQVcsTUFBTTtjQUFJLE9BQU87ZUFDeEQ7Y0FBQyxNQUFNO2NBQWdCLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztlQUN2RDtjQUFDLE1BQU07Y0FBb0IsSUFBSTtjQUFXLE1BQU07Y0FBSSxPQUFPO2VBQzNEO2NBQUMsTUFB
 TTtjQUFhLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztjQUFXLGNBQWM7OztVQUUvRSxRQUFRO1lBQUMsU0FBUzs7VUFDbEIsU0FBUztZQUFDLFFBQVE7O1VBQ2xCLFdBQVc7WUFBQyxTQUFTOztVQUNyQixTQUFTO1lBQUMsU0FBUzs7OztNQUdyQixJQUFJLFFBQUEsZ0JBQUEsTUFBSjtRQUNFLFFBQVEsV0FBVzs7TUFFckIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLE9BQUQ7UUN1QnZCLE9EdEJBLGFBQWE7O01Dd0JmLE9EckJBLGVBQWUsU0FBQyxPQUFEO1FDc0JiLE9EckJFLENBQUEsU0FBQyxPQUFEO1VBQ0EsSUFBQSxPQUFBLFNBQUE7VUFBQSxZQUFZLE1BQU07VUFDbEIsUUFBUSxRQUFRO1VBQ2hCLElBQUcsTUFBTSxRQUFPLFdBQWhCO1lDdUJFLE9EdEJBLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLEtBQUssUUFBUSxNQUNsRSxNQUFNO2lCQUhYO1lBS0UsVUFBVTtZQUNWLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLFNBQVMsUUFBUSxNQUN0RSxNQUFNO1lBQ1QsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQUFRLE1BQ3RFLE1BQU07WUNpQlQsT0RoQkEsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQ
 UFRLE1BQ3RFLE1BQU07O1dBakJWOzs7OztBQ3NDWDtBQzdGQSxRQUFRLE9BQU8sWUFFZCxRQUFRLHNEQUF1QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUM5QixLQUFDLGVBQWUsV0FBQTtJQUNkLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZ0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUSxLQUFLOztJQ25CeEIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7SUFFRCxRQUFRLDJEQUE0QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNuQyxLQUFDLGNBQWMsU0FBQyxlQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxrQkFBa0IsZUFDM0IsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DdEJQLE9EdUJBLFNBQVMsUUFBUSxLQUFLOztJQ3JCeEIsT0R1QkEsU0FBUzs7RUNyQlgsT0R1QkE7O0FDckJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbG
 VcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5ydW4gKCRyb290U2NvcGUpIC0+XHJcbiAgJHJvb3RTY29wZS5zaWR
 lYmFyVmlzaWJsZSA9IGZhbHNlXHJcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XHJcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcclxuICAgICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4udmFsdWUgJ2ZsaW5rQ29uZmlnJywge1xyXG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxyXG59XHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4ucnVuIChKb2JzU2VydmljZSwgTWFpblNlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIC0+XHJcbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cclxuICAgIGFuZ3VsYXIuZXh0ZW5kIGZsaW5rQ29uZmlnLCBjb25maWdcclxuXHJcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXHJcblxyXG4gICAgJGludGVydmFsIC0+XHJcbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcclxuICAgICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gIEhpZ2hjaGFydHMuc2V0T3B0aW9ucyh7XHJcbiAgICBnbG9iYWw6IHtcclxuICAgICAgdXNlVVRDOiBmYWxzZVxyXG4gICAgfVxyXG4gIH0pXHJcblxyXG4gICNcclxuICAjIEdyaWQtbGlnaHQgdGhlbWUgZm9yIEhpZ2hj
 aGFydHMgSlNcclxuICAjIEBhdXRob3IgVG9yc3RlaW4gSG9uc2lcclxuICAjXHJcbiAgIyBUYWtlbiBmcm9tIGh0dHBzOi8vZ2l0aHViLmNvbS9oaWdoc2xpZGUtc29mdHdhcmUvaGlnaGNoYXJ0cy5jb21cclxuICAjXHJcblxyXG5cclxuICBIaWdoY2hhcnRzLmNyZWF0ZUVsZW1lbnQoJ2xpbmsnLCB7XHJcbiAgXHRocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxyXG4gIFx0cmVsOiAnc3R5bGVzaGVldCcsXHJcbiAgXHR0eXBlOiAndGV4dC9jc3MnXHJcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XHJcblxyXG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XHJcbiAgXHRjb2xvcnM6IFtcIiM3Y2I1ZWNcIiwgXCIjZjdhMzVjXCIsIFwiIzkwZWU3ZVwiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCIsIFwiI2ZmMDA2NlwiLCBcIiNlZWFhZWVcIixcclxuICBcdFx0XCIjNTVCRjNCXCIsIFwiI0RGNTM1M1wiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCJdLFxyXG4gIFx0Y2hhcnQ6IHtcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiBudWxsLFxyXG4gIFx0XHRzdHlsZToge1xyXG4gIFx0XHRcdGZvbnRGYW1pbHk6IFwiRG9zaXMsIHNhbnMtc2VyaWZcIlxyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dGl0bGU6IHtcclxuICBcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRmb250U2l6ZTogJzE2cHgnLFxyXG4gIFx0XHRcd
 GZvbnRXZWlnaHQ6ICdib2xkJyxcclxuICBcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dG9vbHRpcDoge1xyXG4gIFx0XHRib3JkZXJXaWR0aDogMCxcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiAncmdiYSgyMTksMjE5LDIxNiwwLjgpJyxcclxuICBcdFx0c2hhZG93OiBmYWxzZVxyXG4gIFx0fSxcclxuICBcdGxlZ2VuZDoge1xyXG4gIFx0XHRpdGVtU3R5bGU6IHtcclxuICBcdFx0XHRmb250V2VpZ2h0OiAnYm9sZCcsXHJcbiAgXHRcdFx0Zm9udFNpemU6ICcxM3B4J1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0eEF4aXM6IHtcclxuICBcdFx0Z3JpZExpbmVXaWR0aDogMSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHR5QXhpczoge1xyXG4gIFx0XHRtaW5vclRpY2tJbnRlcnZhbDogJ2F1dG8nLFxyXG4gIFx0XHR0aXRsZToge1xyXG4gIFx0XHRcdHN0eWxlOiB7XHJcbiAgXHRcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHRcdH1cclxuICBcdFx0fSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHRwbG90T3B0aW9ucz
 oge1xyXG4gIFx0XHRjYW5kbGVzdGljazoge1xyXG4gIFx0XHRcdGxpbmVDb2xvcjogJyM0MDQwNDgnXHJcbiAgXHRcdH1cclxuICBcdH0sXHJcblxyXG4gIFx0YmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xyXG5cclxuICB9O1xyXG5cclxuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoSGlnaGNoYXJ0cy50aGVtZSk7XHJcblxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxyXG4gICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cclxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXHJcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxyXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgIG1haW46XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5
 nLWpvYnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcclxuICBcclxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxyXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgbWFpbjpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxyXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcclxuICAgIHVybDogXCJcIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIixcclxuICAgIHVybDogXCJc
 IlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0Lm92ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLFxyXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcclxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInIFxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXHJcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxyXG4gICAgdXJsOiBcIi97dmVydGV4SWR9XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICB2ZXJ0ZXg6XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4L
 mh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLFxyXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IuZXhjZXB0aW9uc1wiLFxyXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIixcclxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOl
 xyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiYWxsLW1hbmFnZXJcIixcclxuICAgIHVybDogXCIvdGFza21hbmFnZXJzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL2luZGV4Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL3Rhc2ttYW5hZ2VyL3t0YXNrbWFuYWdlcmlkfVwiXHJcbiAgICAgIHZpZXdzOlxyXG4gICAgICAgIG1haW46XHJcbiAgICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YX

<TRUNCATED>

[10/47] flink git commit: [FLINK-2354] [runtime] Remove state changing futures in JobManager

Posted by tr...@apache.org.
[FLINK-2354] [runtime] Remove state changing futures in JobManager

Internal actor states must only be modified within the actor thread.
This avoids all the well-known issues coming with concurrency.

Fix RemoveCachedJob by introducing RemoveJob

Fix JobManagerITCase

Add removeJob which maintains the job in the SubmittedJobGraphStore

Make revokeLeadership not remove the jobs from the state backend

Fix shading problem with curator by hiding CuratorFramework in ChaosMonkeyITCase


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

Branch: refs/heads/master
Commit: c2989f2b1839055858e4b328473d0a8313094ff3
Parents: 73c73e9
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Oct 9 00:50:07 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:52 2015 +0200

----------------------------------------------------------------------
 flink-runtime/pom.xml                           |   4 +-
 .../flink/runtime/jobmanager/JobManager.scala   | 262 +++++++++++--------
 .../runtime/messages/JobManagerMessages.scala   |  17 ++
 ...ManagerSubmittedJobGraphsRecoveryITCase.java |   5 +-
 .../zookeeper/ZooKeeperTestEnvironment.java     |  10 +
 .../src/test/resources/log4j-test.properties    |   2 +-
 .../runtime/jobmanager/JobManagerITCase.scala   |   2 +-
 .../flink-shaded-curator-recipes/pom.xml        |  78 ++++++
 .../flink-shaded-curator-test/pom.xml           |  86 ++++++
 flink-shaded-curator/pom.xml                    |  82 ++----
 flink-tests/pom.xml                             |   7 +
 .../flink/test/recovery/ChaosMonkeyITCase.java  |  15 +-
 .../flink/yarn/YARNHighAvailabilityITCase.java  |  13 +-
 .../flink/yarn/TestingYarnJobManager.scala      |  10 +-
 14 files changed, 404 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index f79c5ed..9db82b2 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -182,7 +182,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-curator</artifactId>
+			<artifactId>flink-shaded-curator-recipes</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		
@@ -417,7 +417,7 @@ under the License.
 						<configuration>
 							<artifactSet>
 								<includes combine.children="append">
-									<include>org.apache.flink:flink-shaded-curator</include>
+									<include>org.apache.flink:flink-shaded-curator-recipes</include>
 								</includes>
 							</artifactSet>
 							<relocations combine.children="append">

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/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 f3e4054..eef28d8 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
@@ -64,7 +64,6 @@ import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMess
 import org.apache.flink.util.{ExceptionUtils, InstantiationUtil, NetUtils}
 
 import scala.collection.JavaConverters._
-import scala.concurrent.ExecutionContext.Implicits.global
 import scala.concurrent._
 import scala.concurrent.duration._
 import scala.concurrent.forkjoin.ForkJoinPool
@@ -127,6 +126,9 @@ class JobManager(
 
   var leaderSessionID: Option[UUID] = None
 
+  /** Futures which have to be completed before terminating the job manager */
+  var futuresToComplete: Option[Seq[Future[Unit]]] = None
+
   /**
    * Run when the job manager is started. Simply logs an informational message.
    * The method also starts the leader election service.
@@ -163,7 +165,16 @@ class JobManager(
   override def postStop(): Unit = {
     log.info(s"Stopping JobManager ${getAddress}.")
 
-    cancelAndClearEverything(new Exception("The JobManager is shutting down."))
+    val newFuturesToComplete = cancelAndClearEverything(
+      new Exception("The JobManager is shutting down."),
+      true)
+
+    implicit val executionContext = context.dispatcher
+
+    val futureToComplete = Future.sequence(
+      futuresToComplete.getOrElse(Seq()) ++ newFuturesToComplete)
+
+    Await.ready(futureToComplete, timeout)
 
     // disconnect the registered task managers
     instanceManager.getAllRegisteredInstances.asScala.foreach {
@@ -235,9 +246,11 @@ class JobManager(
     case RevokeLeadership =>
       log.info(s"JobManager ${self.path.toSerializationFormat} was revoked leadership.")
 
-      future {
-        cancelAndClearEverything(new Exception("JobManager is no longer the leader."))
-      }(context.dispatcher)
+      val newFuturesToComplete = cancelAndClearEverything(
+        new Exception("JobManager is no longer the leader."),
+        false)
+
+      futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) ++ newFuturesToComplete)
 
       // disconnect the registered task managers
       instanceManager.getAllRegisteredInstances.asScala.foreach {
@@ -315,9 +328,15 @@ class JobManager(
       val jobInfo = new JobInfo(client, listeningBehaviour, System.currentTimeMillis(),
         jobGraph.getSessionTimeout)
 
-      future {
-        submitJob(jobGraph, jobInfo)
-      }(context.dispatcher)
+      submitJob(jobGraph, jobInfo)
+
+    case RecoverSubmittedJob(submittedJobGraph) =>
+      if (!currentJobs.contains(submittedJobGraph.getJobId)) {
+        submitJob(
+          submittedJobGraph.getJobGraph(),
+          submittedJobGraph.getJobInfo(),
+          isRecovery = true)
+      }
 
     case RecoverJob(jobId) =>
       future {
@@ -328,19 +347,18 @@ class JobManager(
 
           log.info(s"Attempting to recover job $jobId.")
 
-          val jobGraph = submittedJobGraphs.recoverJobGraph(jobId)
+          val submittedJobGraphOption = submittedJobGraphs.recoverJobGraph(jobId)
 
-          if (jobGraph.isDefined) {
-            if (!leaderElectionService.hasLeadership()) {
-              // we've lost leadership. mission: abort.
-              log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.")
-            }
-            else {
-              recoverJobGraph(jobGraph.get)
-            }
-          }
-          else {
-            log.warn(s"Failed to recover job graph ${jobId}.")
+          submittedJobGraphOption match {
+            case Some(submittedJobGraph) =>
+              if (!leaderElectionService.hasLeadership()) {
+                // we've lost leadership. mission: abort.
+                log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.")
+              }
+              else {
+                self ! decorateMessage(RecoverSubmittedJob(submittedJobGraph))
+              }
+            case None => log.warn(s"Failed to recover job graph $jobId.")
           }
         }
       }(context.dispatcher)
@@ -362,7 +380,10 @@ class JobManager(
           else {
             log.debug(s"Attempting to recover ${jobGraphs.size} job graphs.")
 
-            jobGraphs.foreach(recoverJobGraph(_))
+            jobGraphs.foreach{
+              submittedJobGraph =>
+                self ! decorateMessage(RecoverSubmittedJob(submittedJobGraph))
+            }
           }
         }
       }(context.dispatcher)
@@ -473,7 +494,7 @@ class JobManager(
           if (newJobStatus.isTerminalState()) {
             jobInfo.end = timeStamp
 
-            future {
+            future{
               // TODO If removing the JobGraph from the SubmittedJobGraphsStore fails, the job will
               // linger around and potentially be recovered at a later time. There is nothing we
               // can do about that, but it should be communicated with the Client.
@@ -483,11 +504,11 @@ class JobManager(
                 context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) {
                   // remove only if no activity occurred in the meantime
                   if (lastActivity == jobInfo.lastActive) {
-                    removeJob(jobID)
+                    self ! decorateMessage(RemoveJob(jobID, true))
                   }
-                }
+                }(context.dispatcher)
               } else {
-                removeJob(jobID)
+                self ! decorateMessage(RemoveJob(jobID, true))
               }
 
               // is the client waiting for the job result?
@@ -539,9 +560,7 @@ class JobManager(
             }(context.dispatcher)
           }
         case None =>
-          future {
-            removeJob(jobID)
-          }(context.dispatcher)
+          self ! decorateMessage(RemoveJob(jobID, true))
       }
 
     case ScheduleOrUpdateConsumers(jobId, partitionId) =>
@@ -646,9 +665,7 @@ class JobManager(
     case Heartbeat(instanceID, metricsReport, accumulators) =>
       log.debug(s"Received hearbeat message from $instanceID.")
 
-      Future {
-        updateAccumulators(accumulators)
-      }(context.dispatcher)
+      updateAccumulators(accumulators)
 
       instanceManager.reportHeartBeat(instanceID, metricsReport)
 
@@ -671,11 +688,26 @@ class JobManager(
     case RequestJobManagerStatus =>
       sender() ! decorateMessage(JobManagerStatusAlive)
 
+    case RemoveJob(jobID, clearPersistedJob) =>
+      currentJobs.get(jobID) match {
+        case Some((graph, info)) =>
+            removeJob(graph.getJobID, clearPersistedJob) match {
+              case Some(futureToComplete) =>
+                futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) :+ futureToComplete)
+              case None =>
+            }
+        case None =>
+      }
+
     case RemoveCachedJob(jobID) =>
       currentJobs.get(jobID) match {
         case Some((graph, info)) =>
           if (graph.getState.isTerminalState) {
-            removeJob(graph.getJobID)
+            removeJob(graph.getJobID, true) match {
+              case Some(futureToComplete) =>
+                futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) :+ futureToComplete)
+              case None =>
+            }
           } else {
             // triggers removal upon completion of job
             info.sessionAlive = false
@@ -761,6 +793,7 @@ class JobManager(
               jobGraph.getClasspaths,
               userCodeLoader)
 
+            currentJobs.put(jobGraph.getJobID, (graph, jobInfo))
             graph
         }
 
@@ -878,22 +911,6 @@ class JobManager(
           executionGraph.registerExecutionListener(gateway)
           executionGraph.registerJobStatusListener(gateway)
         }
-
-        if (isRecovery) {
-          executionGraph.restoreLatestCheckpointedState()
-        }
-        else {
-          submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo))
-        }
-
-        // Add the job graph only after everything is finished. Otherwise there can be races in
-        // tests, which check the currentJobs (for example before killing a JM).
-        if (!currentJobs.contains(jobId)) {
-          currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo))
-        }
-
-        // done with submitting the job
-        jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID))
       }
       catch {
         case t: Throwable =>
@@ -916,20 +933,39 @@ class JobManager(
           return
       }
 
-      if (leaderElectionService.hasLeadership) {
-        // There is a small chance that multiple job managers schedule the same job after if they
-        // try to recover at the same time. This will eventually be noticed, but can not be ruled
-        // out from the beginning.
-
-        // NOTE: Scheduling the job for execution is a separate action from the job submission.
-        // The success of submitting the job must be independent from the success of scheduling
-        // the job.
+      // execute the recovery/writing the jobGraph into the SubmittedJobGraphStore asynchronously
+      // because it is a blocking operation
+      future {
         try {
-          log.info(s"Scheduling job $jobId ($jobName).")
+          if (isRecovery) {
+            executionGraph.restoreLatestCheckpointedState()
+          }
+          else {
+            submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo))
+          }
 
-          executionGraph.scheduleForExecution(scheduler)
-        }
-        catch {
+          jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID))
+
+          if (leaderElectionService.hasLeadership) {
+            // There is a small chance that multiple job managers schedule the same job after if
+            // they try to recover at the same time. This will eventually be noticed, but can not be
+            // ruled out from the beginning.
+
+            // NOTE: Scheduling the job for execution is a separate action from the job submission.
+            // The success of submitting the job must be independent from the success of scheduling
+            // the job.
+            log.info(s"Scheduling job $jobId ($jobName).")
+
+            executionGraph.scheduleForExecution(scheduler)
+          } else {
+            // Remove the job graph. Otherwise it will be lingering around and possibly removed from
+            // ZooKeeper by this JM.
+            self ! decorateMessage(RemoveJob(jobId, false))
+
+            log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " +
+              "this. I am not scheduling the job for execution.")
+          }
+        } catch {
           case t: Throwable => try {
             executionGraph.fail(t)
           }
@@ -939,27 +975,6 @@ class JobManager(
             }
           }
         }
-      }
-      else {
-        // Remove the job graph. Otherwise it will be lingering around and possibly removed from
-        // ZooKeeper by this JM.
-        currentJobs.remove(jobId)
-
-        log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " +
-          "this. I am not scheduling the job for execution.")
-      }
-    }
-  }
-
-  /**
-   * Submits the job if it is not already one of our current jobs.
-   *
-   * @param jobGraph Job to recover
-   */
-  private def recoverJobGraph(jobGraph: SubmittedJobGraph): Unit = {
-    if (!currentJobs.contains(jobGraph.getJobId)) {
-      future {
-        submitJob(jobGraph.getJobGraph(), jobGraph.getJobInfo(), isRecovery = true)
       }(context.dispatcher)
     }
   }
@@ -1169,20 +1184,24 @@ class JobManager(
    * might block. Therefore be careful not to block the actor thread.
    *
    * @param jobID ID of the job to remove and archive
+   * @param removeJobFromStateBackend true if the job shall be archived and removed from the state
+   *                            backend
    */
-  private def removeJob(jobID: JobID): Unit = {
-    currentJobs.synchronized {
-      // Don't remove the job yet...
-      currentJobs.get(jobID) match {
-        case Some((eg, _)) =>
-          try {
-            // ...otherwise, we can have lingering resources when there is a  concurrent shutdown
-            // and the ZooKeeper client is closed. Not removing the job immediately allow the
-            // shutdown to release all resources.
-            submittedJobGraphs.removeJobGraph(jobID)
-          } catch {
-            case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t)
-          }
+  private def removeJob(jobID: JobID, removeJobFromStateBackend: Boolean): Option[Future[Unit]] = {
+    // Don't remove the job yet...
+    val futureOption = currentJobs.get(jobID) match {
+      case Some((eg, _)) =>
+        val result = if (removeJobFromStateBackend) {
+          val futureOption = Some(future {
+            try {
+              // ...otherwise, we can have lingering resources when there is a  concurrent shutdown
+              // and the ZooKeeper client is closed. Not removing the job immediately allow the
+              // shutdown to release all resources.
+              submittedJobGraphs.removeJobGraph(jobID)
+            } catch {
+              case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t)
+            }
+          }(context.dispatcher))
 
           try {
             eg.prepareForArchiving()
@@ -1193,9 +1212,15 @@ class JobManager(
               "archiving.", t)
           }
 
-          currentJobs.remove(jobID)
-        case None =>
-      }
+          futureOption
+        } else {
+          None
+        }
+
+        currentJobs.remove(jobID)
+
+        result
+      case None => None
     }
 
     try {
@@ -1204,6 +1229,8 @@ class JobManager(
       case t: Throwable =>
         log.error(s"Could not properly unregister job $jobID form the library cache.", t)
     }
+
+    futureOption
   }
 
   /** Fails all currently running jobs and empties the list of currently running jobs. If the
@@ -1211,26 +1238,35 @@ class JobManager(
     *
     * @param cause Cause for the cancelling.
     */
-  private def cancelAndClearEverything(cause: Throwable) {
-    for ((jobID, (eg, jobInfo)) <- currentJobs) {
-      try {
-        submittedJobGraphs.removeJobGraph(jobID)
-      }
-      catch {
-        case t: Throwable => {
-          log.error("Error during submitted job graph clean up.", t)
+  private def cancelAndClearEverything(
+      cause: Throwable,
+      removeJobFromStateBackend: Boolean)
+    : Seq[Future[Unit]] = {
+    val futures = for ((jobID, (eg, jobInfo)) <- currentJobs) yield {
+      future {
+        if (removeJobFromStateBackend) {
+          try {
+            submittedJobGraphs.removeJobGraph(jobID)
+          }
+          catch {
+            case t: Throwable => {
+              log.error("Error during submitted job graph clean up.", t)
+            }
+          }
         }
-      }
 
-      eg.fail(cause)
+        eg.fail(cause)
 
-      if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) {
-        jobInfo.client ! decorateMessage(
-          Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause)))
-      }
+        if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) {
+          jobInfo.client ! decorateMessage(
+            Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause)))
+        }
+      }(context.dispatcher)
     }
 
     currentJobs.clear()
+
+    futures.toSeq
   }
 
   override def grantLeadership(newLeaderSessionID: UUID): Unit = {
@@ -1285,7 +1321,9 @@ class JobManager(
       case accumulatorEvent =>
         currentJobs.get(accumulatorEvent.getJobID) match {
           case Some((jobGraph, jobInfo)) =>
-            jobGraph.updateAccumulators(accumulatorEvent)
+            future {
+              jobGraph.updateAccumulators(accumulatorEvent)
+            }(context.dispatcher)
           case None =>
           // ignore accumulator values for old job
         }

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index d776622..8097bdc 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGra
 import org.apache.flink.runtime.instance.{InstanceID, Instance}
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID
 import org.apache.flink.runtime.jobgraph.{IntermediateDataSetID, JobGraph, JobStatus, JobVertexID}
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph
 import org.apache.flink.runtime.util.SerializedThrowable
 
 import scala.collection.JavaConverters._
@@ -73,6 +74,14 @@ object JobManagerMessages {
   case class RecoverJob(jobId: JobID) extends RequiresLeaderSessionID
 
   /**
+   * Triggers the submission of the recovered job
+   *
+   * @param submittedJobGraph Contains the submitted JobGraph and the associated JobInfo
+   */
+  case class RecoverSubmittedJob(submittedJobGraph: SubmittedJobGraph)
+    extends RequiresLeaderSessionID
+
+  /**
    * Triggers recovery of all available jobs.
    */
   case class RecoverAllJobs() extends RequiresLeaderSessionID
@@ -286,6 +295,14 @@ object JobManagerMessages {
    */
   case class JobNotFound(jobID: JobID) extends JobResponse with JobStatusResponse
 
+  /** Triggers the removal of the job with the given job ID
+    *
+    * @param jobID
+    * @param removeJobFromStateBackend true if the job has properly finished
+    */
+  case class RemoveJob(jobID: JobID, removeJobFromStateBackend: Boolean = true)
+    extends RequiresLeaderSessionID
+
   /**
    * Removes the job belonging to the job identifier from the job manager and archives it.
    * @param jobID The job identifier

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
index ac250bd..e6156e5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
@@ -194,7 +194,9 @@ public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger {
 			JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING,
 					leadingJobManager, deadline.timeLeft());
 
-			// Make sure that the **non-leading** JM has actually removed the job graph from her
+			log.info("Wait that the non-leader removes the submitted job.");
+
+			// Make sure that the **non-leading** JM has actually removed the job graph from its
 			// local state.
 			boolean success = false;
 			while (!success && deadline.hasTimeLeft()) {
@@ -205,6 +207,7 @@ public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger {
 					success = true;
 				}
 				else {
+					log.info(((JobManagerMessages.CurrentJobStatus)jobStatusResponse).status().toString());
 					Thread.sleep(100);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
index 7ae89d1..94e1988 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
@@ -26,6 +26,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 
+import java.util.List;
+
 /**
  * Simple ZooKeeper and CuratorFramework setup for tests.
  */
@@ -111,6 +113,14 @@ public class ZooKeeperTestEnvironment {
 		return client;
 	}
 
+	public String getClientNamespace() {
+		return client.getNamespace();
+	}
+
+	public List<String> getChildren(String path) throws Exception {
+		return client.getChildren().forPath(path);
+	}
+
 	/**
 	 * Creates a new client for the started ZooKeeper server/cluster.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties
index 76b237e..1ca02aa 100644
--- a/flink-runtime/src/test/resources/log4j-test.properties
+++ b/flink-runtime/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=OFF, console
+log4j.rootLogger=INFO, console
 
 # -----------------------------------------------------------------------------
 # Console (use 'console')

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
index 3a252f8..0f800c9 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
@@ -657,7 +657,7 @@ class JobManagerITCase(_system: ActorSystem)
           jm.tell(SubmitJob(jobGraph2, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph2.getJobID))
 
-          // job stil running
+          // job still running
           jm.tell(RemoveCachedJob(jobGraph2.getJobID), self)
 
           expectMsgType[JobResultSuccess]

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
new file mode 100644
index 0000000..c0a2adc
--- /dev/null
+++ b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-shaded-curator</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-shaded-curator-recipes</artifactId>
+	<name>flink-shaded-curator-recipes</name>
+
+	<packaging>jar</packaging>
+
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-recipes</artifactId>
+			<version>${curator.version}</version>
+		</dependency>
+
+		<!-- Use Flink's Guava version here to avoid too many guava versions in Flink -->
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet combine.self="override">
+								<excludes>
+									<exclude>log4j</exclude>
+									<exclude>org.slf4j:slf4j-log4j12</exclude>
+								</excludes>
+							</artifactSet>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-shaded-curator/flink-shaded-curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/flink-shaded-curator-test/pom.xml b/flink-shaded-curator/flink-shaded-curator-test/pom.xml
new file mode 100644
index 0000000..2700c0c
--- /dev/null
+++ b/flink-shaded-curator/flink-shaded-curator-test/pom.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-shaded-curator</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-shaded-curator-test</artifactId>
+	<name>flink-shaded-curator-test</name>
+
+	<packaging>jar</packaging>
+
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+		</dependency>
+
+		<!-- Use Flink's Guava version here to avoid too many guava versions in Flink -->
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet combine.self="override">
+								<excludes>
+									<exclude>log4j</exclude>
+									<exclude>org.slf4j:slf4j-log4j12</exclude>
+								</excludes>
+								<includes combine.children="append">
+									<include>org.apache.curator:curator-test</include>
+								</includes>
+							</artifactSet>
+							<relocations combine.children="append">
+								<relocation>
+									<pattern>org.apache.curator</pattern>
+									<shadedPattern>org.apache.flink.shaded.org.apache.curator</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-shaded-curator/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/pom.xml b/flink-shaded-curator/pom.xml
index ac62cc8..29d6461 100644
--- a/flink-shaded-curator/pom.xml
+++ b/flink-shaded-curator/pom.xml
@@ -1,22 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!--
-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.
--->
+  ~ 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.
+  -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
@@ -29,50 +28,13 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
+	<modules>
+        <module>flink-shaded-curator-recipes</module>
+		<module>flink-shaded-curator-test</module>
+	</modules>
+
 	<artifactId>flink-shaded-curator</artifactId>
 	<name>flink-shaded-curator</name>
 
-	<packaging>jar</packaging>
-
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-recipes</artifactId>
-			<version>${curator.version}</version>
-		</dependency>
-
-		<!-- Use Flink's Guava version here to avoid too many guava versions in Flink -->
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<artifactSet combine.self="override">
-								<excludes>
-									<exclude>log4j</exclude>
-									<exclude>org.slf4j:slf4j-log4j12</exclude>
-								</excludes>
-							</artifactSet>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
+	<packaging>pom</packaging>
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 0dd20b1..b9bae6f 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -148,6 +148,13 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-curator-test</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
 			<groupId>org.scalatest</groupId>
 			<artifactId>scalatest_${scala.binary.version}</artifactId>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
index a0c8312..2cdf83c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
@@ -523,22 +523,19 @@ public class ChaosMonkeyITCase {
 	}
 
 	private void checkCleanRecoveryState(Configuration config) throws Exception {
-		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+		LOG.info("Checking " + ZooKeeper.getClientNamespace() +
 				ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
-		List<String> jobGraphs = ZooKeeper.getClient().getChildren()
-				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
+		List<String> jobGraphs = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
 		assertEquals("Unclean job graphs: " + jobGraphs, 0, jobGraphs.size());
 
-		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+		LOG.info("Checking " + ZooKeeper.getClientNamespace() +
 				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
-		List<String> checkpoints = ZooKeeper.getClient().getChildren()
-				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
+		List<String> checkpoints = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
 		assertEquals("Unclean checkpoints: " + checkpoints, 0, checkpoints.size());
 
-		LOG.info("Checking " + ZooKeeper.getClient().getNamespace() +
+		LOG.info("Checking " + ZooKeeper.getClientNamespace() +
 				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
-		List<String> checkpointCounter = ZooKeeper.getClient().getChildren()
-				.forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
+		List<String> checkpointCounter = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
 		assertEquals("Unclean checkpoint counter: " + checkpointCounter, 0, checkpointCounter.size());
 
 		LOG.info("ZooKeeper state is clean");

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 94d0a81..a05621a 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -22,6 +22,7 @@ import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.testkit.JavaTestKit;
 import org.apache.curator.test.TestingServer;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -37,7 +38,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
@@ -53,6 +56,9 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 
 	private static final int numberApplicationAttempts = 10;
 
+	@Rule
+	public TemporaryFolder tmp = new TemporaryFolder();
+
 	@BeforeClass
 	public static void setup() {
 		actorSystem = AkkaUtils.createDefaultActorSystem();
@@ -102,9 +108,14 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 		String confDirPath = System.getenv("FLINK_CONF_DIR");
 		flinkYarnClient.setConfigurationDirectory(confDirPath);
 
+		String fsStateHandlePath = tmp.getRoot().getPath();
+
 		flinkYarnClient.setFlinkConfigurationObject(GlobalConfiguration.getConfiguration());
 		flinkYarnClient.setDynamicPropertiesEncoded("recovery.mode=zookeeper@@ha.zookeeper.quorum=" +
-			zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts);
+			zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts +
+			"@@" + ConfigConstants.STATE_BACKEND + "=FILESYSTEM" +
+			"@@" + ConfigConstants.STATE_BACKEND_FS_DIR + "=" + fsStateHandlePath + "/checkpoints" +
+			"@@" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "=" + fsStateHandlePath + "/recovery");
 		flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
 
 		AbstractFlinkYarnCluster yarnCluster = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/c2989f2b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala
index 83d1f3c..fa70039 100644
--- a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala
+++ b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala
@@ -21,8 +21,10 @@ package org.apache.flink.yarn
 import akka.actor.ActorRef
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.StreamingMode
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.instance.InstanceManager
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.leaderelection.LeaderElectionService
 import org.apache.flink.runtime.testingUtils.TestingJobManagerLike
@@ -60,7 +62,9 @@ class TestingYarnJobManager(
     delayBetweenRetries: Long,
     timeout: FiniteDuration,
     mode: StreamingMode,
-    leaderElectionService: LeaderElectionService)
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory)
   extends YarnJobManager(
     flinkConfiguration,
     executionContext,
@@ -72,7 +76,9 @@ class TestingYarnJobManager(
     delayBetweenRetries,
     timeout,
     mode,
-    leaderElectionService)
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory)
   with TestingJobManagerLike {
 
   override val taskManagerRunnerClass = classOf[TestingYarnTaskManagerRunner]


[02/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
deleted file mode 100644
index f517f83..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
+++ /dev/null
@@ -1,391 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.checkpointing;
-
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.util.Collector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * A simple test that runs a streaming topology with checkpointing enabled.
- *
- * The test triggers a failure after a while and verifies that, after completion, the
- * state defined with either the {@link OperatorState} or the {@link Checkpointed}
- * interface reflects the "exactly once" semantics.
- * 
- * The test throttles the input until at least two checkpoints are completed, to make sure that
- * the recovery does not fall back to "square one" (which would naturally lead to correct
- * results without testing the checkpointing).
- */
-@SuppressWarnings("serial")
-public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StateCheckpoinedITCase.class);
-
-	final long NUM_STRINGS = 10_000_000L;
-
-	/**
-	 * Runs the following program:
-	 *
-	 * <pre>
-	 *     [ (source)->(filter)] -> [ (map) -> (map) ] -> [ (groupBy/reduce)->(sink) ]
-	 * </pre>
-	 */
-	@Override
-	public void testProgram(StreamExecutionEnvironment env) {
-		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
-
-		final long failurePosMin = (long) (0.4 * NUM_STRINGS / PARALLELISM);
-		final long failurePosMax = (long) (0.7 * NUM_STRINGS / PARALLELISM);
-
-		final long failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
-
-		env.enableCheckpointing(200);
-
-		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS));
-
-		stream
-				// first vertex, chained to the source
-				// this filter throttles the flow until at least one checkpoint
-				// is complete, to make sure this program does not run without 
-				.filter(new StringRichFilterFunction())
-
-						// -------------- seconds vertex - one-to-one connected ----------------
-				.map(new StringPrefixCountRichMapFunction())
-				.startNewChain()
-				.map(new StatefulCounterFunction())
-
-						// -------------- third vertex - reducer and the sink ----------------
-				.partitionByHash("prefix")
-				.flatMap(new OnceFailingAggregator(failurePos))
-				.addSink(new ValidatingSink());
-	}
-
-	@Override
-	public void postSubmit() {
-		
-		//assertTrue("Test inconclusive: failure occurred before first checkpoint",
-		//		OnceFailingAggregator.wasCheckpointedBeforeFailure);
-		if(!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
-			LOG.warn("Test inconclusive: failure occurred before first checkpoint");
-		}
-		
-		long filterSum = 0;
-		for (long l : StringRichFilterFunction.counts) {
-			filterSum += l;
-		}
-
-		long mapSum = 0;
-		for (long l : StringPrefixCountRichMapFunction.counts) {
-			mapSum += l;
-		}
-
-		long countSum = 0;
-		for (long l : StatefulCounterFunction.counts) {
-			countSum += l;
-		}
-
-		// verify that we counted exactly right
-		assertEquals(NUM_STRINGS, filterSum);
-		assertEquals(NUM_STRINGS, mapSum);
-		assertEquals(NUM_STRINGS, countSum);
-
-		for (Map<Character, Long> map : ValidatingSink.maps) {
-			for (Long count : map.values()) {
-				assertEquals(NUM_STRINGS / 40, count.longValue());
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Custom Functions
-	// --------------------------------------------------------------------------------------------
-	
-	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String> 
-			implements CheckpointedAsynchronously<Integer>
-	{
-		private final long numElements;
-
-		private int index;
-
-		private volatile boolean isRunning = true;
-		
-		
-		StringGeneratingSourceFunction(long numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			final Object lockingObject = ctx.getCheckpointLock();
-
-			final Random rnd = new Random();
-			final StringBuilder stringBuilder = new StringBuilder();
-			
-			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
-			
-			if (index == 0) {
-				index = getRuntimeContext().getIndexOfThisSubtask();
-			}
-
-			while (isRunning && index < numElements) {
-				char first = (char) ((index % 40) + 40);
-
-				stringBuilder.setLength(0);
-				stringBuilder.append(first);
-
-				String result = randomString(stringBuilder, rnd);
-
-				synchronized (lockingObject) {
-					index += step;
-					ctx.collect(result);
-				}
-			}
-		}
-		
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		private static String randomString(StringBuilder bld, Random rnd) {
-			final int len = rnd.nextInt(10) + 5;
-
-			for (int i = 0; i < len; i++) {
-				char next = (char) (rnd.nextInt(20000) + 33);
-				bld.append(next);
-			}
-
-			return bld.toString();
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			index = state;
-		}
-	}
-
-	private static class StringRichFilterFunction extends RichFilterFunction<String> 
-			implements Checkpointed<Long> {
-
-		static final long[] counts = new long[PARALLELISM];
-		
-		private long count;
-		
-		@Override
-		public boolean filter(String value) throws Exception {
-			count++;
-			return value.length() < 100; // should be always true
-		}
-
-		@Override
-		public void close() {
-			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
-		}
-
-		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
-		}
-
-		@Override
-		public void restoreState(Long state) {
-			count = state;
-		}
-	}
-
-	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> 
-			implements CheckpointedAsynchronously<Long> {
-		
-		static final long[] counts = new long[PARALLELISM];
-
-		private long count;
-		
-		@Override
-		public PrefixCount map(String value) {
-			count++;
-			return new PrefixCount(value.substring(0, 1), value, 1L);
-		}
-
-		@Override
-		public void close() {
-			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
-		}
-
-		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
-		}
-
-		@Override
-		public void restoreState(Long state) {
-			count = state;
-		}
-	}
-	
-	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
-		implements Checkpointed<Long> {
-
-		static final long[] counts = new long[PARALLELISM];
-		
-		private long count;
-
-		@Override
-		public PrefixCount map(PrefixCount value) throws Exception {
-			count++;
-			return value;
-		}
-
-		@Override
-		public void close() throws IOException {
-			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
-		}
-
-		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
-		}
-
-		@Override
-		public void restoreState(Long state) {
-			count = state;
-		}
-	}
-	
-	private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount> 
-		implements Checkpointed<HashMap<String, PrefixCount>>, CheckpointNotifier {
-
-		static boolean wasCheckpointedBeforeFailure = false;
-		
-		private static volatile boolean hasFailed = false;
-
-		private final HashMap<String, PrefixCount> aggregationMap = new HashMap<String, PrefixCount>();
-		
-		private long failurePos;
-		private long count;
-		
-		private boolean wasCheckpointed;
-		
-
-		OnceFailingAggregator(long failurePos) {
-			this.failurePos = failurePos;
-		}
-		
-		@Override
-		public void open(Configuration parameters) {
-			count = 0;
-		}
-
-		@Override
-		public void flatMap(PrefixCount value, Collector<PrefixCount> out) throws Exception {
-			count++;
-			if (!hasFailed && count >= failurePos && getRuntimeContext().getIndexOfThisSubtask() == 1) {
-				wasCheckpointedBeforeFailure = wasCheckpointed;
-				hasFailed = true;
-				throw new Exception("Test Failure");
-			}
-			
-			PrefixCount curr = aggregationMap.get(value.prefix);
-			if (curr == null) {
-				aggregationMap.put(value.prefix, value);
-				out.collect(value);
-			}
-			else {
-				curr.count += value.count;
-				out.collect(curr);
-			}
-		}
-
-		@Override
-		public HashMap<String, PrefixCount> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return aggregationMap;
-		}
-
-		@Override
-		public void restoreState(HashMap<String, PrefixCount> state) {
-			aggregationMap.putAll(state);
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) {
-			this.wasCheckpointed = true;
-		}
-	}
-
-	private static class ValidatingSink extends RichSinkFunction<PrefixCount> 
-			implements Checkpointed<HashMap<Character, Long>> {
-
-		@SuppressWarnings("unchecked")
-		private static Map<Character, Long>[] maps = (Map<Character, Long>[]) new Map<?, ?>[PARALLELISM];
-		
-		private HashMap<Character, Long> counts = new HashMap<Character, Long>();
-
-		@Override
-		public void invoke(PrefixCount value) {
-			Character first = value.prefix.charAt(0);
-			Long previous = counts.get(first);
-			if (previous == null) {
-				counts.put(first, value.count);
-			} else {
-				counts.put(first, Math.max(previous, value.count));
-			}
-		}
-
-		@Override
-		public void close() throws Exception {
-			maps[getRuntimeContext().getIndexOfThisSubtask()] = counts;
-		}
-
-		@Override
-		public HashMap<Character, Long> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return counts;
-		}
-
-		@Override
-		public void restoreState(HashMap<Character, Long> state) {
-			counts.putAll(state);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
new file mode 100644
index 0000000..d7c06f6
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
@@ -0,0 +1,391 @@
+/*
+ * 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.test.checkpointing;
+
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * A simple test that runs a streaming topology with checkpointing enabled.
+ *
+ * The test triggers a failure after a while and verifies that, after completion, the
+ * state defined with either the {@link OperatorState} or the {@link Checkpointed}
+ * interface reflects the "exactly once" semantics.
+ * 
+ * The test throttles the input until at least two checkpoints are completed, to make sure that
+ * the recovery does not fall back to "square one" (which would naturally lead to correct
+ * results without testing the checkpointing).
+ */
+@SuppressWarnings("serial")
+public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StateCheckpointedITCase.class);
+
+	final long NUM_STRINGS = 10_000_000L;
+
+	/**
+	 * Runs the following program:
+	 *
+	 * <pre>
+	 *     [ (source)->(filter)] -> [ (map) -> (map) ] -> [ (groupBy/reduce)->(sink) ]
+	 * </pre>
+	 */
+	@Override
+	public void testProgram(StreamExecutionEnvironment env) {
+		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
+
+		final long failurePosMin = (long) (0.4 * NUM_STRINGS / PARALLELISM);
+		final long failurePosMax = (long) (0.7 * NUM_STRINGS / PARALLELISM);
+
+		final long failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
+
+		env.enableCheckpointing(200);
+
+		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS));
+
+		stream
+				// first vertex, chained to the source
+				// this filter throttles the flow until at least one checkpoint
+				// is complete, to make sure this program does not run without 
+				.filter(new StringRichFilterFunction())
+
+						// -------------- seconds vertex - one-to-one connected ----------------
+				.map(new StringPrefixCountRichMapFunction())
+				.startNewChain()
+				.map(new StatefulCounterFunction())
+
+						// -------------- third vertex - reducer and the sink ----------------
+				.partitionByHash("prefix")
+				.flatMap(new OnceFailingAggregator(failurePos))
+				.addSink(new ValidatingSink());
+	}
+
+	@Override
+	public void postSubmit() {
+		
+		//assertTrue("Test inconclusive: failure occurred before first checkpoint",
+		//		OnceFailingAggregator.wasCheckpointedBeforeFailure);
+		if(!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
+			LOG.warn("Test inconclusive: failure occurred before first checkpoint");
+		}
+		
+		long filterSum = 0;
+		for (long l : StringRichFilterFunction.counts) {
+			filterSum += l;
+		}
+
+		long mapSum = 0;
+		for (long l : StringPrefixCountRichMapFunction.counts) {
+			mapSum += l;
+		}
+
+		long countSum = 0;
+		for (long l : StatefulCounterFunction.counts) {
+			countSum += l;
+		}
+
+		// verify that we counted exactly right
+		assertEquals(NUM_STRINGS, filterSum);
+		assertEquals(NUM_STRINGS, mapSum);
+		assertEquals(NUM_STRINGS, countSum);
+
+		for (Map<Character, Long> map : ValidatingSink.maps) {
+			for (Long count : map.values()) {
+				assertEquals(NUM_STRINGS / 40, count.longValue());
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Custom Functions
+	// --------------------------------------------------------------------------------------------
+	
+	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String> 
+			implements CheckpointedAsynchronously<Integer>
+	{
+		private final long numElements;
+
+		private int index;
+
+		private volatile boolean isRunning = true;
+		
+		
+		StringGeneratingSourceFunction(long numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			final Object lockingObject = ctx.getCheckpointLock();
+
+			final Random rnd = new Random();
+			final StringBuilder stringBuilder = new StringBuilder();
+			
+			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
+			
+			if (index == 0) {
+				index = getRuntimeContext().getIndexOfThisSubtask();
+			}
+
+			while (isRunning && index < numElements) {
+				char first = (char) ((index % 40) + 40);
+
+				stringBuilder.setLength(0);
+				stringBuilder.append(first);
+
+				String result = randomString(stringBuilder, rnd);
+
+				synchronized (lockingObject) {
+					index += step;
+					ctx.collect(result);
+				}
+			}
+		}
+		
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		private static String randomString(StringBuilder bld, Random rnd) {
+			final int len = rnd.nextInt(10) + 5;
+
+			for (int i = 0; i < len; i++) {
+				char next = (char) (rnd.nextInt(20000) + 33);
+				bld.append(next);
+			}
+
+			return bld.toString();
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+			return index;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+			index = state;
+		}
+	}
+
+	private static class StringRichFilterFunction extends RichFilterFunction<String> 
+			implements Checkpointed<Long> {
+
+		static final long[] counts = new long[PARALLELISM];
+		
+		private long count;
+		
+		@Override
+		public boolean filter(String value) throws Exception {
+			count++;
+			return value.length() < 100; // should be always true
+		}
+
+		@Override
+		public void close() {
+			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+			return count;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			count = state;
+		}
+	}
+
+	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> 
+			implements CheckpointedAsynchronously<Long> {
+		
+		static final long[] counts = new long[PARALLELISM];
+
+		private long count;
+		
+		@Override
+		public PrefixCount map(String value) {
+			count++;
+			return new PrefixCount(value.substring(0, 1), value, 1L);
+		}
+
+		@Override
+		public void close() {
+			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+			return count;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			count = state;
+		}
+	}
+	
+	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
+		implements Checkpointed<Long> {
+
+		static final long[] counts = new long[PARALLELISM];
+		
+		private long count;
+
+		@Override
+		public PrefixCount map(PrefixCount value) throws Exception {
+			count++;
+			return value;
+		}
+
+		@Override
+		public void close() throws IOException {
+			counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
+		}
+
+		@Override
+		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+			return count;
+		}
+
+		@Override
+		public void restoreState(Long state) {
+			count = state;
+		}
+	}
+	
+	private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount> 
+		implements Checkpointed<HashMap<String, PrefixCount>>, CheckpointNotifier {
+
+		static boolean wasCheckpointedBeforeFailure = false;
+		
+		private static volatile boolean hasFailed = false;
+
+		private final HashMap<String, PrefixCount> aggregationMap = new HashMap<String, PrefixCount>();
+		
+		private long failurePos;
+		private long count;
+		
+		private boolean wasCheckpointed;
+		
+
+		OnceFailingAggregator(long failurePos) {
+			this.failurePos = failurePos;
+		}
+		
+		@Override
+		public void open(Configuration parameters) {
+			count = 0;
+		}
+
+		@Override
+		public void flatMap(PrefixCount value, Collector<PrefixCount> out) throws Exception {
+			count++;
+			if (!hasFailed && count >= failurePos && getRuntimeContext().getIndexOfThisSubtask() == 1) {
+				wasCheckpointedBeforeFailure = wasCheckpointed;
+				hasFailed = true;
+				throw new Exception("Test Failure");
+			}
+			
+			PrefixCount curr = aggregationMap.get(value.prefix);
+			if (curr == null) {
+				aggregationMap.put(value.prefix, value);
+				out.collect(value);
+			}
+			else {
+				curr.count += value.count;
+				out.collect(curr);
+			}
+		}
+
+		@Override
+		public HashMap<String, PrefixCount> snapshotState(long checkpointId, long checkpointTimestamp) {
+			return aggregationMap;
+		}
+
+		@Override
+		public void restoreState(HashMap<String, PrefixCount> state) {
+			aggregationMap.putAll(state);
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) {
+			this.wasCheckpointed = true;
+		}
+	}
+
+	private static class ValidatingSink extends RichSinkFunction<PrefixCount> 
+			implements Checkpointed<HashMap<Character, Long>> {
+
+		@SuppressWarnings("unchecked")
+		private static Map<Character, Long>[] maps = (Map<Character, Long>[]) new Map<?, ?>[PARALLELISM];
+		
+		private HashMap<Character, Long> counts = new HashMap<Character, Long>();
+
+		@Override
+		public void invoke(PrefixCount value) {
+			Character first = value.prefix.charAt(0);
+			Long previous = counts.get(first);
+			if (previous == null) {
+				counts.put(first, value.count);
+			} else {
+				counts.put(first, Math.max(previous, value.count));
+			}
+		}
+
+		@Override
+		public void close() throws Exception {
+			maps[getRuntimeContext().getIndexOfThisSubtask()] = counts;
+		}
+
+		@Override
+		public HashMap<Character, Long> snapshotState(long checkpointId, long checkpointTimestamp) {
+			return counts;
+		}
+
+		@Override
+		public void restoreState(HashMap<Character, Long> state) {
+			counts.putAll(state);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java
new file mode 100644
index 0000000..ba5ff1c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java
@@ -0,0 +1,289 @@
+/*
+ * 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.test.recovery;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerProcess;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Option;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTempDirectory;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+/**
+ * Verify behaviour in case of JobManager process failure during job execution.
+ *
+ * <p>The test works with multiple job managers processes by spawning JVMs.
+ *
+ * <p>Initially, it starts two TaskManager (2 slots each) and two JobManager JVMs.
+ *
+ * <p>It submits a program with parallelism 4 and waits until all tasks are brought up.
+ * Coordination between the test and the tasks happens via checking for the existence of
+ * temporary files. It then kills the leading JobManager process. The recovery should restart the
+ * tasks on the new JobManager.
+ *
+ * <p>This follows the same structure as {@link AbstractTaskManagerProcessFailureRecoveryTest}.
+ */
+public abstract class AbstractJobManagerProcessFailureRecoveryITCase extends TestLogger {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	private static final File FileStateBackendBasePath;
+
+	static {
+		try {
+			FileStateBackendBasePath = CommonTestUtils.createTempDirectory();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error in test setup. Could not create directory.", e);
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (ZooKeeper != null) {
+			ZooKeeper.shutdown();
+		}
+
+		if (FileStateBackendBasePath != null) {
+			FileUtils.deleteDirectory(FileStateBackendBasePath);
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		ZooKeeper.deleteAll();
+
+		FileUtils.cleanDirectory(FileStateBackendBasePath);
+	}
+
+	protected static final String READY_MARKER_FILE_PREFIX = "ready_";
+	protected static final String FINISH_MARKER_FILE_PREFIX = "finish_";
+	protected static final String PROCEED_MARKER_FILE = "proceed";
+
+	protected static final int PARALLELISM = 4;
+
+	/**
+	 * Test program with JobManager failure.
+	 *
+	 * @param zkQuorum ZooKeeper quorum to connect to
+	 * @param coordinateDir Coordination directory
+	 * @throws Exception
+	 */
+	public abstract void testJobManagerFailure(String zkQuorum, File coordinateDir) throws Exception;
+
+	@Test
+	public void testJobManagerProcessFailure() throws Exception {
+		// Config
+		final int numberOfJobManagers = 2;
+		final int numberOfTaskManagers = 2;
+		final int numberOfSlotsPerTaskManager = 2;
+
+		assertEquals(PARALLELISM, numberOfTaskManagers * numberOfSlotsPerTaskManager);
+
+		// Setup
+		// Test actor system
+		ActorSystem testActorSystem;
+
+		// Job managers
+		final JobManagerProcess[] jmProcess = new JobManagerProcess[numberOfJobManagers];
+
+		// Task managers
+		final ActorSystem[] tmActorSystem = new ActorSystem[numberOfTaskManagers];
+
+		// Leader election service
+		LeaderRetrievalService leaderRetrievalService = null;
+
+		// Coordination between the processes goes through a directory
+		File coordinateTempDir = null;
+
+		try {
+			final Deadline deadline = TestTimeOut.fromNow();
+
+			// Coordination directory
+			coordinateTempDir = createTempDirectory();
+
+			// Job Managers
+			Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+					ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());
+
+			// Start first process
+			jmProcess[0] = new JobManagerProcess(0, config);
+			jmProcess[0].createAndStart();
+
+			// Task manager configuration
+			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
+
+			// Start the task manager process
+			for (int i = 0; i < numberOfTaskManagers; i++) {
+				tmActorSystem[i] = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+				TaskManager.startTaskManagerComponentsAndActor(
+						config, tmActorSystem[i], "localhost",
+						Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
+						false, StreamingMode.STREAMING, TaskManager.class);
+			}
+
+			// Test actor system
+			testActorSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+
+			jmProcess[0].getActorRef(testActorSystem, deadline.timeLeft());
+
+			// Leader listener
+			TestingListener leaderListener = new TestingListener();
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService.start(leaderListener);
+
+			// Initial submission
+			leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+			String leaderAddress = leaderListener.getAddress();
+			UUID leaderId = leaderListener.getLeaderSessionID();
+
+			// Get the leader ref
+			ActorRef leaderRef = AkkaUtils.getActorRef(leaderAddress, testActorSystem, deadline.timeLeft());
+			ActorGateway leaderGateway = new AkkaActorGateway(leaderRef, leaderId);
+
+			// Wait for all task managers to connect to the leading job manager
+			JobManagerActorTestUtils.waitForTaskManagers(numberOfTaskManagers, leaderGateway,
+					deadline.timeLeft());
+
+			final File coordinateDirClosure = coordinateTempDir;
+			final Throwable[] errorRef = new Throwable[1];
+
+			// we trigger program execution in a separate thread
+			Thread programTrigger = new Thread("Program Trigger") {
+				@Override
+				public void run() {
+					try {
+						testJobManagerFailure(ZooKeeper.getConnectString(), coordinateDirClosure);
+					}
+					catch (Throwable t) {
+						t.printStackTrace();
+						errorRef[0] = t;
+					}
+				}
+			};
+
+			//start the test program
+			programTrigger.start();
+
+			// wait until all marker files are in place, indicating that all tasks have started
+			AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(coordinateTempDir,
+					READY_MARKER_FILE_PREFIX, PARALLELISM, deadline.timeLeft().toMillis());
+
+			// Kill one of the job managers and trigger recovery
+			jmProcess[0].destroy();
+
+			jmProcess[1] = new JobManagerProcess(1, config);
+			jmProcess[1].createAndStart();
+
+			jmProcess[1].getActorRef(testActorSystem, deadline.timeLeft());
+
+			// we create the marker file which signals the program functions tasks that they can complete
+			AbstractTaskManagerProcessFailureRecoveryTest.touchFile(new File(coordinateTempDir, PROCEED_MARKER_FILE));
+
+			programTrigger.join(deadline.timeLeft().toMillis());
+
+			// We wait for the finish marker file. We don't wait for the program trigger, because
+			// we submit in detached mode.
+			AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(coordinateTempDir,
+					FINISH_MARKER_FILE_PREFIX, 1, deadline.timeLeft().toMillis());
+
+			// check that the program really finished
+			assertFalse("The program did not finish in time", programTrigger.isAlive());
+
+			// check whether the program encountered an error
+			if (errorRef[0] != null) {
+				Throwable error = errorRef[0];
+				error.printStackTrace();
+				fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+
+			for (JobManagerProcess p : jmProcess) {
+				if (p != null) {
+					p.printProcessLog();
+				}
+			}
+
+			fail(e.getMessage());
+		}
+		finally {
+			for (int i = 0; i < numberOfTaskManagers; i++) {
+				if (tmActorSystem[i] != null) {
+					tmActorSystem[i].shutdown();
+				}
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			for (JobManagerProcess jmProces : jmProcess) {
+				if (jmProces != null) {
+					jmProces.destroy();
+				}
+			}
+
+			// Delete coordination directory
+			if (coordinateTempDir != null) {
+				try {
+					FileUtils.deleteDirectory(coordinateTempDir);
+				}
+				catch (Throwable ignored) {
+				}
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java
deleted file mode 100644
index 7e16baf..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java
+++ /dev/null
@@ -1,444 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.recovery;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.StreamingMode;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobmanager.JobManager;
-import org.apache.flink.runtime.jobmanager.MemoryArchivist;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.taskmanager.TaskManager;
-import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
-import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-/**
- * Abstract base for tests verifying the behavior of the recovery in the
- * case when a TaskManager fails (process is killed) in the middle of a job execution.
- *
- * The test works with multiple task managers processes by spawning JVMs.
- * Initially, it starts a JobManager in process and two TaskManagers JVMs with
- * 2 task slots each.
- * It submits a program with parallelism 4 and waits until all tasks are brought up.
- * Coordination between the test and the tasks happens via checking for the
- * existence of temporary files. It then starts another TaskManager, which is
- * guaranteed to remain empty (all tasks are already deployed) and kills one of
- * the original task managers. The recovery should restart the tasks on the new TaskManager.
- */
-public abstract class AbstractProcessFailureRecoveryTest extends TestLogger {
-
-	protected static final String READY_MARKER_FILE_PREFIX = "ready_";
-	protected static final String PROCEED_MARKER_FILE = "proceed";
-
-	protected static final int PARALLELISM = 4;
-
-	@Test
-	public void testTaskManagerProcessFailure() {
-
-		final StringWriter processOutput1 = new StringWriter();
-		final StringWriter processOutput2 = new StringWriter();
-		final StringWriter processOutput3 = new StringWriter();
-
-		ActorSystem jmActorSystem = null;
-		Process taskManagerProcess1 = null;
-		Process taskManagerProcess2 = null;
-		Process taskManagerProcess3 = null;
-
-		File coordinateTempDir = null;
-
-		try {
-			// check that we run this test only if the java command
-			// is available on this machine
-			String javaCommand = getJavaCommandPath();
-			if (javaCommand == null) {
-				System.out.println("---- Skipping Process Failure test : Could not find java executable ----");
-				return;
-			}
-
-			// create a logging file for the process
-			File tempLogFile = File.createTempFile(getClass().getSimpleName() + "-", "-log4j.properties");
-			tempLogFile.deleteOnExit();
-			CommonTestUtils.printLog4jDebugConfig(tempLogFile);
-
-			// coordination between the processes goes through a directory
-			coordinateTempDir = createTempDirectory();
-
-			// find a free port to start the JobManager
-			final int jobManagerPort = NetUtils.getAvailablePort();
-
-			// start a JobManager
-			Tuple2<String, Object> localAddress = new Tuple2<String, Object>("localhost", jobManagerPort);
-
-			Configuration jmConfig = new Configuration();
-			jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms");
-			jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s");
-			jmConfig.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
-			jmConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "10 s");
-
-			jmActorSystem = AkkaUtils.createActorSystem(jmConfig, new Some<Tuple2<String, Object>>(localAddress));
-			ActorRef jmActor = JobManager.startJobManagerActors(
-				jmConfig,
-				jmActorSystem,
-				StreamingMode.STREAMING,
-				JobManager.class,
-				MemoryArchivist.class)._1();
-
-			// the TaskManager java command
-			String[] command = new String[] {
-					javaCommand,
-					"-Dlog.level=DEBUG",
-					"-Dlog4j.configuration=file:" + tempLogFile.getAbsolutePath(),
-					"-Xms80m", "-Xmx80m",
-					"-classpath", getCurrentClasspath(),
-					TaskManagerProcessEntryPoint.class.getName(),
-					String.valueOf(jobManagerPort)
-			};
-
-			// start the first two TaskManager processes
-			taskManagerProcess1 = new ProcessBuilder(command).start();
-			new PipeForwarder(taskManagerProcess1.getErrorStream(), processOutput1);
-			taskManagerProcess2 = new ProcessBuilder(command).start();
-			new PipeForwarder(taskManagerProcess2.getErrorStream(), processOutput2);
-
-			// we wait for the JobManager to have the two TaskManagers available
-			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)
-			waitUntilNumTaskManagersAreRegistered(jmActor, 2, 120000);
-
-			// the program will set a marker file in each of its parallel tasks once they are ready, so that
-			// this coordinating code is aware of this.
-			// the program will very slowly consume elements until the marker file (later created by the
-			// test driver code) is present
-			final File coordinateDirClosure = coordinateTempDir;
-			final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-
-			// we trigger program execution in a separate thread
-			Thread programTrigger = new Thread("Program Trigger") {
-				@Override
-				public void run() {
-					try {
-						testProgram(jobManagerPort, coordinateDirClosure);
-					}
-					catch (Throwable t) {
-						t.printStackTrace();
-						errorRef.set(t);
-					}
-				}
-			};
-
-			//start the test program
-			programTrigger.start();
-
-			// wait until all marker files are in place, indicating that all tasks have started
-			// max 20 seconds
-			if (!waitForMarkerFiles(coordinateTempDir, PARALLELISM, 120000)) {
-				// check if the program failed for some reason
-				if (errorRef.get() != null) {
-					Throwable error = errorRef.get();
-					error.printStackTrace();
-					fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage());
-				}
-				else {
-					// no error occurred, simply a timeout
-					fail("The tasks were not started within time (" + 120000 + "msecs)");
-				}
-			}
-
-			// start the third TaskManager
-			taskManagerProcess3 = new ProcessBuilder(command).start();
-			new PipeForwarder(taskManagerProcess3.getErrorStream(), processOutput3);
-
-			// we wait for the third TaskManager to register
-			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)
-			waitUntilNumTaskManagersAreRegistered(jmActor, 3, 120000);
-
-			// kill one of the previous TaskManagers, triggering a failure and recovery
-			taskManagerProcess1.destroy();
-			taskManagerProcess1 = null;
-
-			// we create the marker file which signals the program functions tasks that they can complete
-			touchFile(new File(coordinateTempDir, PROCEED_MARKER_FILE));
-
-			// wait for at most 5 minutes for the program to complete
-			programTrigger.join(300000);
-
-			// check that the program really finished
-			assertFalse("The program did not finish in time", programTrigger.isAlive());
-
-			// check whether the program encountered an error
-			if (errorRef.get() != null) {
-				Throwable error = errorRef.get();
-				error.printStackTrace();
-				fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage());
-			}
-
-			// all seems well :-)
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			printProcessLog("TaskManager 1", processOutput1.toString());
-			printProcessLog("TaskManager 2", processOutput2.toString());
-			printProcessLog("TaskManager 3", processOutput3.toString());
-			fail(e.getMessage());
-		}
-		catch (Error e) {
-			e.printStackTrace();
-			printProcessLog("TaskManager 1", processOutput1.toString());
-			printProcessLog("TaskManager 2", processOutput2.toString());
-			printProcessLog("TaskManager 3", processOutput3.toString());
-			throw e;
-		}
-		finally {
-			if (taskManagerProcess1 != null) {
-				taskManagerProcess1.destroy();
-			}
-			if (taskManagerProcess2 != null) {
-				taskManagerProcess2.destroy();
-			}
-			if (taskManagerProcess3 != null) {
-				taskManagerProcess3.destroy();
-			}
-			if (jmActorSystem != null) {
-				jmActorSystem.shutdown();
-			}
-			if (coordinateTempDir != null) {
-				try {
-					FileUtils.deleteDirectory(coordinateTempDir);
-				}
-				catch (Throwable t) {
-					// we can ignore this
-				}
-			}
-		}
-	}
-
-	/**
-	 * The test program should be implemented here in a form of a separate thread.
-	 * This provides a solution for checking that it has been terminated.
-	 *
-	 * @param jobManagerPort The port for submitting the topology to the local cluster
-	 * @param coordinateDir TaskManager failure will be triggered only after processes
-	 *                             have successfully created file under this directory
-	 */
-	public abstract void testProgram(int jobManagerPort, File coordinateDir) throws Exception;
-
-
-	protected void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelay)
-			throws Exception
-	{
-		final long deadline = System.currentTimeMillis() + maxDelay;
-		while (true) {
-			long remaining = deadline - System.currentTimeMillis();
-			if (remaining <= 0) {
-				fail("The TaskManagers did not register within the expected time (" + maxDelay + "msecs)");
-			}
-
-			FiniteDuration timeout = new FiniteDuration(remaining, TimeUnit.MILLISECONDS);
-
-			try {
-				Future<?> result = Patterns.ask(jobManager,
-						JobManagerMessages.getRequestNumberRegisteredTaskManager(),
-						new Timeout(timeout));
-				Integer numTMs = (Integer) Await.result(result, timeout);
-				if (numTMs == numExpected) {
-					break;
-				}
-			}
-			catch (TimeoutException e) {
-				// ignore and retry
-			}
-			catch (ClassCastException e) {
-				fail("Wrong response: " + e.getMessage());
-			}
-		}
-	}
-
-	protected static void printProcessLog(String processName, String log) {
-		if (log == null || log.length() == 0) {
-			return;
-		}
-
-		System.out.println("-----------------------------------------");
-		System.out.println(" BEGIN SPAWNED PROCESS LOG FOR " + processName);
-		System.out.println("-----------------------------------------");
-		System.out.println(log);
-		System.out.println("-----------------------------------------");
-		System.out.println("		END SPAWNED PROCESS LOG");
-		System.out.println("-----------------------------------------");
-	}
-
-	protected static File createTempDirectory() throws IOException {
-		File tempDir = new File(System.getProperty("java.io.tmpdir"));
-
-		for (int i = 0; i < 10; i++) {
-			File dir = new File(tempDir, UUID.randomUUID().toString());
-			if (!dir.exists() && dir.mkdirs()) {
-				return dir;
-			}
-			System.err.println("Could not use temporary directory " + dir.getAbsolutePath());
-		}
-
-		throw new IOException("Could not create temporary file directory");
-	}
-
-	protected static void touchFile(File file) throws IOException {
-		if (!file.exists()) {
-			new FileOutputStream(file).close();
-		}
-		if (!file.setLastModified(System.currentTimeMillis())) {
-			throw new IOException("Could not touch the file.");
-		}
-	}
-
-	protected static boolean waitForMarkerFiles(File basedir, int num, long timeout) {
-		long now = System.currentTimeMillis();
-		final long deadline = now + timeout;
-
-
-		while (now < deadline) {
-			boolean allFound = true;
-
-			for (int i = 0; i < num; i++) {
-				File nextToCheck = new File(basedir, READY_MARKER_FILE_PREFIX + i);
-				if (!nextToCheck.exists()) {
-					allFound = false;
-					break;
-				}
-			}
-
-			if (allFound) {
-				return true;
-			}
-			else {
-				// not all found, wait for a bit
-				try {
-					Thread.sleep(10);
-				}
-				catch (InterruptedException e) {
-					throw new RuntimeException(e);
-				}
-
-				now = System.currentTimeMillis();
-			}
-		}
-
-		return false;
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * The entry point for the TaskManager JVM. Simply configures and runs a TaskManager.
-	 */
-	public static class TaskManagerProcessEntryPoint {
-
-		private static final Logger LOG = LoggerFactory.getLogger(TaskManagerProcessEntryPoint.class);
-
-		public static void main(String[] args) {
-			try {
-				int jobManagerPort = Integer.parseInt(args[0]);
-
-				Configuration cfg = new Configuration();
-				cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-				cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
-				cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
-				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
-				cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
-
-				TaskManager.selectNetworkInterfaceAndRunTaskManager(cfg, StreamingMode.STREAMING, TaskManager.class);
-
-				// wait forever
-				Object lock = new Object();
-				synchronized (lock) {
-					lock.wait();
-				}
-			}
-			catch (Throwable t) {
-				LOG.error("Failed to start TaskManager process", t);
-				System.exit(1);
-			}
-		}
-	}
-
-	/**
-	 * Utility class to read the output of a process stream and forward it into a StringWriter.
-	 */
-	protected static class PipeForwarder extends Thread {
-
-		private final StringWriter target;
-		private final InputStream source;
-
-		public PipeForwarder(InputStream source, StringWriter target) {
-			super("Pipe Forwarder");
-			setDaemon(true);
-
-			this.source = source;
-			this.target = target;
-
-			start();
-		}
-
-		@Override
-		public void run() {
-			try {
-				int next;
-				while ((next = source.read()) != -1) {
-					target.write(next);
-				}
-			}
-			catch (IOException e) {
-				// terminate
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
new file mode 100644
index 0000000..c02fa6c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
@@ -0,0 +1,397 @@
+/*
+ * 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.test.recovery;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.util.NetUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+/**
+ * Abstract base for tests verifying the behavior of the recovery in the
+ * case when a TaskManager fails (process is killed) in the middle of a job execution.
+ *
+ * The test works with multiple task managers processes by spawning JVMs.
+ * Initially, it starts a JobManager in process and two TaskManagers JVMs with
+ * 2 task slots each.
+ * It submits a program with parallelism 4 and waits until all tasks are brought up.
+ * Coordination between the test and the tasks happens via checking for the
+ * existence of temporary files. It then starts another TaskManager, which is
+ * guaranteed to remain empty (all tasks are already deployed) and kills one of
+ * the original task managers. The recovery should restart the tasks on the new TaskManager.
+ */
+public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends TestLogger {
+
+	protected static final String READY_MARKER_FILE_PREFIX = "ready_";
+	protected static final String PROCEED_MARKER_FILE = "proceed";
+	protected static final String FINISH_MARKER_FILE_PREFIX = "finish_";
+
+	protected static final int PARALLELISM = 4;
+
+	@Test
+	public void testTaskManagerProcessFailure() {
+
+		final StringWriter processOutput1 = new StringWriter();
+		final StringWriter processOutput2 = new StringWriter();
+		final StringWriter processOutput3 = new StringWriter();
+
+		ActorSystem jmActorSystem = null;
+		Process taskManagerProcess1 = null;
+		Process taskManagerProcess2 = null;
+		Process taskManagerProcess3 = null;
+
+		File coordinateTempDir = null;
+
+		try {
+			// check that we run this test only if the java command
+			// is available on this machine
+			String javaCommand = getJavaCommandPath();
+			if (javaCommand == null) {
+				System.out.println("---- Skipping Process Failure test : Could not find java executable ----");
+				return;
+			}
+
+			// create a logging file for the process
+			File tempLogFile = File.createTempFile(getClass().getSimpleName() + "-", "-log4j.properties");
+			tempLogFile.deleteOnExit();
+			CommonTestUtils.printLog4jDebugConfig(tempLogFile);
+
+			// coordination between the processes goes through a directory
+			coordinateTempDir = CommonTestUtils.createTempDirectory();
+
+			// find a free port to start the JobManager
+			final int jobManagerPort = NetUtils.getAvailablePort();
+
+			// start a JobManager
+			Tuple2<String, Object> localAddress = new Tuple2<String, Object>("localhost", jobManagerPort);
+
+			Configuration jmConfig = new Configuration();
+			jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms");
+			jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s");
+			jmConfig.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
+			jmConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "10 s");
+
+			jmActorSystem = AkkaUtils.createActorSystem(jmConfig, new Some<Tuple2<String, Object>>(localAddress));
+			ActorRef jmActor = JobManager.startJobManagerActors(
+				jmConfig,
+				jmActorSystem,
+				StreamingMode.STREAMING,
+				JobManager.class,
+				MemoryArchivist.class)._1();
+
+			// the TaskManager java command
+			String[] command = new String[] {
+					javaCommand,
+					"-Dlog.level=DEBUG",
+					"-Dlog4j.configuration=file:" + tempLogFile.getAbsolutePath(),
+					"-Xms80m", "-Xmx80m",
+					"-classpath", getCurrentClasspath(),
+					TaskManagerProcessEntryPoint.class.getName(),
+					String.valueOf(jobManagerPort)
+			};
+
+			// start the first two TaskManager processes
+			taskManagerProcess1 = new ProcessBuilder(command).start();
+			new CommonTestUtils.PipeForwarder(taskManagerProcess1.getErrorStream(), processOutput1);
+			taskManagerProcess2 = new ProcessBuilder(command).start();
+			new CommonTestUtils.PipeForwarder(taskManagerProcess2.getErrorStream(), processOutput2);
+
+			// we wait for the JobManager to have the two TaskManagers available
+			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)
+			waitUntilNumTaskManagersAreRegistered(jmActor, 2, 120000);
+
+			// the program will set a marker file in each of its parallel tasks once they are ready, so that
+			// this coordinating code is aware of this.
+			// the program will very slowly consume elements until the marker file (later created by the
+			// test driver code) is present
+			final File coordinateDirClosure = coordinateTempDir;
+			final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+			// we trigger program execution in a separate thread
+			Thread programTrigger = new Thread("Program Trigger") {
+				@Override
+				public void run() {
+					try {
+						testTaskManagerFailure(jobManagerPort, coordinateDirClosure);
+					}
+					catch (Throwable t) {
+						t.printStackTrace();
+						errorRef.set(t);
+					}
+				}
+			};
+
+			//start the test program
+			programTrigger.start();
+
+			// wait until all marker files are in place, indicating that all tasks have started
+			// max 20 seconds
+			if (!waitForMarkerFiles(coordinateTempDir, READY_MARKER_FILE_PREFIX, PARALLELISM, 120000)) {
+				// check if the program failed for some reason
+				if (errorRef.get() != null) {
+					Throwable error = errorRef.get();
+					error.printStackTrace();
+					fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage());
+				}
+				else {
+					// no error occurred, simply a timeout
+					fail("The tasks were not started within time (" + 120000 + "msecs)");
+				}
+			}
+
+			// start the third TaskManager
+			taskManagerProcess3 = new ProcessBuilder(command).start();
+			new CommonTestUtils.PipeForwarder(taskManagerProcess3.getErrorStream(), processOutput3);
+
+			// we wait for the third TaskManager to register
+			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)
+			waitUntilNumTaskManagersAreRegistered(jmActor, 3, 120000);
+
+			// kill one of the previous TaskManagers, triggering a failure and recovery
+			taskManagerProcess1.destroy();
+			taskManagerProcess1 = null;
+
+			// we create the marker file which signals the program functions tasks that they can complete
+			touchFile(new File(coordinateTempDir, PROCEED_MARKER_FILE));
+
+			// wait for at most 5 minutes for the program to complete
+			programTrigger.join(300000);
+
+			// check that the program really finished
+			assertFalse("The program did not finish in time", programTrigger.isAlive());
+
+			// check whether the program encountered an error
+			if (errorRef.get() != null) {
+				Throwable error = errorRef.get();
+				error.printStackTrace();
+				fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage());
+			}
+
+			// all seems well :-)
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			printProcessLog("TaskManager 1", processOutput1.toString());
+			printProcessLog("TaskManager 2", processOutput2.toString());
+			printProcessLog("TaskManager 3", processOutput3.toString());
+			fail(e.getMessage());
+		}
+		catch (Error e) {
+			e.printStackTrace();
+			printProcessLog("TaskManager 1", processOutput1.toString());
+			printProcessLog("TaskManager 2", processOutput2.toString());
+			printProcessLog("TaskManager 3", processOutput3.toString());
+			throw e;
+		}
+		finally {
+			if (taskManagerProcess1 != null) {
+				taskManagerProcess1.destroy();
+			}
+			if (taskManagerProcess2 != null) {
+				taskManagerProcess2.destroy();
+			}
+			if (taskManagerProcess3 != null) {
+				taskManagerProcess3.destroy();
+			}
+			if (jmActorSystem != null) {
+				jmActorSystem.shutdown();
+			}
+			if (coordinateTempDir != null) {
+				try {
+					FileUtils.deleteDirectory(coordinateTempDir);
+				}
+				catch (Throwable t) {
+					// we can ignore this
+				}
+			}
+		}
+	}
+
+	/**
+	 * The test program should be implemented here in a form of a separate thread.
+	 * This provides a solution for checking that it has been terminated.
+	 *
+	 * @param jobManagerPort The port for submitting the topology to the local cluster
+	 * @param coordinateDir TaskManager failure will be triggered only after processes
+	 *                             have successfully created file under this directory
+	 */
+	public abstract void testTaskManagerFailure(int jobManagerPort, File coordinateDir) throws Exception;
+
+
+	protected void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelay)
+			throws Exception
+	{
+		final long deadline = System.currentTimeMillis() + maxDelay;
+		while (true) {
+			long remaining = deadline - System.currentTimeMillis();
+			if (remaining <= 0) {
+				fail("The TaskManagers did not register within the expected time (" + maxDelay + "msecs)");
+			}
+
+			FiniteDuration timeout = new FiniteDuration(remaining, TimeUnit.MILLISECONDS);
+
+			try {
+				Future<?> result = Patterns.ask(jobManager,
+						JobManagerMessages.getRequestNumberRegisteredTaskManager(),
+						new Timeout(timeout));
+				Integer numTMs = (Integer) Await.result(result, timeout);
+				if (numTMs == numExpected) {
+					break;
+				}
+			}
+			catch (TimeoutException e) {
+				// ignore and retry
+			}
+			catch (ClassCastException e) {
+				fail("Wrong response: " + e.getMessage());
+			}
+		}
+	}
+
+	protected static void printProcessLog(String processName, String log) {
+		if (log == null || log.length() == 0) {
+			return;
+		}
+
+		System.out.println("-----------------------------------------");
+		System.out.println(" BEGIN SPAWNED PROCESS LOG FOR " + processName);
+		System.out.println("-----------------------------------------");
+		System.out.println(log);
+		System.out.println("-----------------------------------------");
+		System.out.println("		END SPAWNED PROCESS LOG");
+		System.out.println("-----------------------------------------");
+	}
+
+	protected static void touchFile(File file) throws IOException {
+		if (!file.exists()) {
+			new FileOutputStream(file).close();
+		}
+		if (!file.setLastModified(System.currentTimeMillis())) {
+			throw new IOException("Could not touch the file.");
+		}
+	}
+
+	protected static boolean waitForMarkerFiles(File basedir, String prefix, int num, long timeout) {
+		long now = System.currentTimeMillis();
+		final long deadline = now + timeout;
+
+
+		while (now < deadline) {
+			boolean allFound = true;
+
+			for (int i = 0; i < num; i++) {
+				File nextToCheck = new File(basedir, prefix + i);
+				if (!nextToCheck.exists()) {
+					allFound = false;
+					break;
+				}
+			}
+
+			if (allFound) {
+				return true;
+			}
+			else {
+				// not all found, wait for a bit
+				try {
+					Thread.sleep(10);
+				}
+				catch (InterruptedException e) {
+					throw new RuntimeException(e);
+				}
+
+				now = System.currentTimeMillis();
+			}
+		}
+
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * The entry point for the TaskManager JVM. Simply configures and runs a TaskManager.
+	 */
+	public static class TaskManagerProcessEntryPoint {
+
+		private static final Logger LOG = LoggerFactory.getLogger(TaskManagerProcessEntryPoint.class);
+
+		public static void main(String[] args) {
+			try {
+				int jobManagerPort = Integer.parseInt(args[0]);
+
+				Configuration cfg = new Configuration();
+				cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+				cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
+				cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
+				cfg.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
+				cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
+
+				TaskManager.selectNetworkInterfaceAndRunTaskManager(cfg, StreamingMode.STREAMING, TaskManager.class);
+
+				// wait forever
+				Object lock = new Object();
+				synchronized (lock) {
+					lock.wait();
+				}
+			}
+			catch (Throwable t) {
+				LOG.error("Failed to start TaskManager process", t);
+				System.exit(1);
+			}
+		}
+	}
+
+}


[39/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
deleted file mode 100644
index da6b75b..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
+++ /dev/null
@@ -1,33 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th>Key</th>
-      <th>Value</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="entry in jobmanager.config | orderBy: 'key'">
-      <td>{{entry.key}}</td>
-      <td>{{entry.value}}</td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
deleted file mode 100644
index 02c2f47..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
+++ /dev/null
@@ -1,33 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Job Manager</div>
-</nav>
-<nav class="navbar navbar-default navbar-fixed-top navbar-main-additional">
-  <ul class="nav nav-tabs">
-    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
-    <li ui-sref-active="active"><a ui-sref=".log">Logs</a></li>
-    <li ui-sref-active="active"><a ui-sref=".stdout">Stdout</a></li>
-  </ul>
-</nav>
-<div id="content-inner" class="has-navbar-main-additional">
-  <div ui-view="details"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
deleted file mode 100644
index df6a817..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">
-        <div class="row">
-          <div class="col-xs-10">Job Manager Output</div>
-          <div class="col-xs-1 text-right"><a ng-click="reloadData()" class="show-pointer"><i class="fa fa-refresh"></i></a></div>
-          <div class="col-xs-1 text-left"><a href="jobmanager/stdout"><i class="fa fa-download"></i></a></div>
-        </div>
-      </th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td colspan="2">
-        <pre>{{jobmanager.stdout}}</pre>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html b/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
deleted file mode 100644
index b76278d..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
+++ /dev/null
@@ -1,53 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Completed Jobs</div>
-</nav>
-<div id="content-inner">
-  <table class="table table-hover table-clickable">
-    <thead>
-      <tr>
-        <th>Start Time</th>
-        <th>End Time</th>
-        <th>Duration</th>
-        <th>Job Name</th>
-        <th>Job ID</th>
-        <th>Tasks</th>
-        <th>Status</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job.duration}} ms</td>
-        <td>{{job.name}}</td>
-        <td>{{job.jid}}</td>
-        <td class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-        </td>
-        <td> 
-          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-        </td>
-      </tr>
-    </tbody>
-  </table>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
deleted file mode 100644
index a7a5d9d..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
+++ /dev/null
@@ -1,57 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table ng-if="job['execution-config']" class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">Execution configuration</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Execution mode</td>
-      <td>{{ job['execution-config']['execution-mode'] }}</td>
-    </tr>
-    <tr>
-      <td>Max. number of execution retries</td>
-      <td>{{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }}</td>
-    </tr>
-    <tr>
-      <td>Job parallelism</td>
-      <td>{{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }}</td>
-    </tr>
-    <tr>
-      <td>Object reuse mode</td>
-      <td>{{ job['execution-config']['object-reuse-mode'] }}</td>
-    </tr>
-  </tbody>
-</table>
-<table ng-if="job['execution-config']['user-config']" class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">User configuration</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="property in job['execution-config']['user-config']">
-      <td>{{property.name}}</td>
-      <td table-property="table-property" value="property.value"></td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
deleted file mode 100644
index a5f6676..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
+++ /dev/null
@@ -1,38 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="exceptions['root-exception']" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">Root exception</div>
-  </div>
-  <div class="panel-body">
-    <pre class="exception">{{ exceptions['root-exception'] }}</pre>
-  </div>
-</div>
-<div ng-repeat="exception in exceptions['all-exceptions']" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <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>
-  <div class="panel-body">
-    <pre class="exception">{{ exception.exception }}</pre>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
deleted file mode 100644
index 9d3e171..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
+++ /dev/null
@@ -1,48 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">
-    <indicator-primary status="{{job.state}}"></indicator-primary>{{ job.name }}
-  </div>
-  <div class="navbar-info first last hidden-xs hidden-sm">{{ job.jid }}</div>
-  <div class="navbar-info first last">
-    <div class="label-group">
-      <bs-label status="{{status}}" ng-repeat="(status, value) in job['status-counts']">{{value}}</bs-label>
-    </div>
-  </div>
-  <div class="navbar-info first last hidden-xs hidden-sm">{{ job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}<span ng-if="job['end-time'] &gt; -1">
-      - 
-      {{ job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></div>
-  <div ng-if="job.duration &gt; -1" class="navbar-info last first">{{job.duration}} ms</div>
-</nav>
-<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
-  <ul class="nav nav-tabs">
-    <li ui-sref-active="active"><a ui-sref=".plan.overview">Plan</a></li>
-    <li ui-sref-active="active"><a ui-sref=".statistics">Job Accumulators / Statistics</a></li>
-    <li ui-sref-active="active"><a ui-sref=".timeline">Timeline</a></li>
-    <li ui-sref-active="active"><a ui-sref=".exceptions">Exceptions</a></li>
-    <li ui-sref-active="active"><a ui-sref=".properties">Properties</a></li>
-    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
-  </ul>
-</nav>
-<div id="content-inner" class="has-navbar-main-additional">
-  <div ui-view="details"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
deleted file mode 100644
index f2c4143..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
+++ /dev/null
@@ -1,31 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
-</div>
-<div ng-if="plan" class="panel panel-default panel-multi">
-  <nav class="navbar navbar-default navbar-secondary-additional">
-    <ul class="nav nav-tabs">
-      <li ui-sref-active="active"><a ui-sref=".overview({nodeid: nodeid})">Overview</a></li>
-      <li ui-sref-active="active"><a ui-sref=".accumulators({nodeid: nodeid})">Accumulators</a></li>
-    </ul>
-  </nav>
-  <div ui-view="node-details" class="panel-body clean"></div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
deleted file mode 100644
index 8de3921..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-body-hover table-clickable table-activable">
-  <thead>
-    <tr>
-      <th>Name</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="v.id == nodeid || changeNode(v.id)">
-    <tr ng-if="v.type == 'regular'">
-      <td>{{ v.name | humanizeText }}</td>
-      <td> 
-        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
-      </td>
-    </tr>
-    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
-      <td colspan="10">
-        <div ng-include=" 'partials/jobs/job.plan.node.accumulators.html' "></div>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
deleted file mode 100644
index 1706d3e..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
+++ /dev/null
@@ -1,60 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-body-hover table-clickable table-activable">
-  <thead>
-    <tr>
-      <th>Start Time</th>
-      <th>End Time</th>
-      <th>Duration</th>
-      <th>Name</th>
-      <th>Bytes read</th>
-      <th>Records read</th>
-      <th>Bytes written</th>
-      <th>Records written</th>
-      <th>Tasks</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="changeNode(v.id)">
-    <tr ng-if="v.type == 'regular'">
-      <td><span ng-if="v['start-time'] &gt; -1">{{ v['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="v['end-time'] &gt; -1">{{ v['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="v.duration &gt; -1">{{ v.duration }} ms</span></td>
-      <td class="td-long">{{ v.name | humanizeText }}</td>
-      <td>{{ v.metrics['read-bytes'] }}</td>
-      <td>{{ v.metrics['read-records'] }}</td>
-      <td>{{ v.metrics['write-bytes'] }}</td>
-      <td>{{ v.metrics['write-records'] }}</td>
-      <td>
-        <div class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(index, status) in stateList">{{v.tasks[status]}}</bs-label>
-        </div>
-      </td>
-      <td> 
-        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
-      </td>
-    </tr>
-    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
-      <td colspan="10">
-        <div ng-include=" 'partials/jobs/job.plan.node.subtasks.html' "></div>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
deleted file mode 100644
index e7dcf2c..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
+++ /dev/null
@@ -1,68 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="accumulators.length == 0">
-  <p><i>No accumulators</i></p>
-</div>
-<div ng-if="accumulators &amp;&amp; accumulators.length &gt; 0">
-  <table class="table table-hover table-clickable table-activable table-inner">
-    <thead>
-      <tr>
-        <th>Name</th>
-        <th>Type</th>
-        <th>Value</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="accumulator in accumulators">
-        <td width="30%">{{ accumulator.name }}</td>
-        <td width="30%">{{ accumulator.type }}</td>
-        <td width="30%">{{ accumulator.value }}</td>
-      </tr>
-    </tbody>
-  </table>
-  <div ng-if="!nodeUnfolded"><a ng-click="toggleFold()" class="btn btn-default">
-      Show subtasks
-       <i class="fa fa-chevron-down"></i></a><a ng-click="deactivateNode(); $event.stopPropagation()" title="Fold" class="btn btn-default pull-right"><i class="fa fa-chevron-up"></i></a></div>
-  <div ng-if="nodeUnfolded &amp;&amp; subtaskAccumulators &amp;&amp; subtaskAccumulators.length &gt; 0"><a ng-click="toggleFold()" class="btn btn-default">
-      Hide subtasks
-       <i class="fa fa-chevron-up"></i></a>
-    <table class="table table-hover table-clickable table-activable table-inner">
-      <thead>
-        <tr>
-          <th>Name</th>
-          <th>Type</th>
-          <th>Value</th>
-        </tr>
-      </thead>
-      <tbody ng-if="subtask['user-accumulators'] &amp;&amp; subtask['user-accumulators'].length &gt; 0" ng-repeat="subtask in subtaskAccumulators">
-        <tr>
-          <td colwidth="3">
-            <div class="small-label">({{ subtask.subtask }}) {{ subtask.host }}, attempt: {{ subtask.attempt + 1 }}</div>
-          </td>
-        </tr>
-        <tr ng-repeat="accumulator in subtask['user-accumulators']">
-          <td width="30%">{{ accumulator.name }}</td>
-          <td width="30%">{{ accumulator.type }}</td>
-          <td width="30%">{{ accumulator.value }}</td>
-        </tr>
-      </tbody>
-    </table>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
deleted file mode 100644
index 40b16bc..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
+++ /dev/null
@@ -1,52 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-
--->
-<table ng-if="subtasks" class="table table-hover table-clickable table-activable table-inner">
-  <thead>
-    <tr>
-      <th>Start Time</th>
-      <th>End Time</th>
-      <th>Duration</th>
-      <th>Bytes read</th>
-      <th>Records read</th>
-      <th>Bytes written</th>
-      <th>Records written</th>
-      <th>Attempt</th>
-      <th>Host</th>
-      <th>Status</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr ng-repeat="subtask in subtasks">
-      <td><span ng-if="subtask['start-time'] &gt; -1">{{ subtask['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="subtask['end-time'] &gt; -1">{{ subtask['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
-      <td><span ng-if="subtask.duration &gt; -1">{{ subtask.duration }} ms</span></td>
-      <td><span ng-if="subtask.metrics['read-bytes'] &gt; -1">{{ subtask.metrics['read-bytes'] }}</span></td>
-      <td><span ng-if="subtask.metrics['read-records'] &gt; -1">{{ subtask.metrics['read-records'] }}</span></td>
-      <td><span ng-if="subtask.metrics['write-bytes'] &gt; -1">{{ subtask.metrics['write-bytes'] }}</span></td>
-      <td><span ng-if="subtask.metrics['write-records'] &gt; -1">{{ subtask.metrics['write-records'] }}</span></td>
-      <td>{{ subtask.attempt + 1 }}</td>
-      <td>{{ subtask.host }}</td>
-      <td> 
-        <bs-label status="{{subtask.status}}">{{subtask.status}}</bs-label>
-      </td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
deleted file mode 100644
index 907afd3..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
+++ /dev/null
@@ -1,140 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
-</div>
-<div ng-if="node" class="panel panel-default">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">{{ node.description | humanizeText }}</div>
-  </div>
-  <div class="panel-body clean">
-    <div class="row">
-      <div class="col-sm-6 col-md-4">
-        <table ng-if="node.optimizer_properties.global_properties" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Global Data Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.global_properties">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-        <table ng-if="node.optimizer_properties.local_properties" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Local Data Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.local_properties">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-        <div class="visible-xs visible-sm">
-          <table class="table table-properties">
-            <thead>
-              <tr>
-                <th colspan="2">Properties</th>
-              </tr>
-            </thead>
-            <tbody>
-              <tr>
-                <td>Operator</td>
-                <td table-property="table-property" value="node.operator_strategy"></td>
-              </tr>
-              <tr>
-                <td>Parallelism</td>
-                <td table-property="table-property" value="node.parallelism"></td>
-              </tr>
-            </tbody>
-          </table>
-        </div>
-      </div>
-      <div class="hidden-sm col-md-4">
-        <table class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Properties</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr>
-              <td>Operator</td>
-              <td table-property="table-property" value="node.operator_strategy"></td>
-            </tr>
-            <tr>
-              <td>Parallelism</td>
-              <td table-property="table-property" value="node.parallelism"></td>
-            </tr>
-          </tbody>
-        </table>
-        <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Size Estimates</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.estimates">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-      </div>
-      <div class="col-sm-6 col-md-4">
-        <div class="visible-xs visible-sm">
-          <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
-            <thead>
-              <tr>
-                <th colspan="2">Size Estimates</th>
-              </tr>
-            </thead>
-            <tbody>
-              <tr ng-repeat="property in node.optimizer_properties.estimates">
-                <td>{{property.name}}</td>
-                <td table-property="table-property" value="property.value"></td>
-              </tr>
-            </tbody>
-          </table>
-        </div>
-        <table ng-if="node.optimizer_properties.costs" class="table table-properties">
-          <thead>
-            <tr>
-              <th colspan="2">Cost Estimates</th>
-            </tr>
-          </thead>
-          <tbody>
-            <tr ng-repeat="property in node.optimizer_properties.costs">
-              <td>{{property.name}}</td>
-              <td table-property="table-property" value="property.value"></td>
-            </tr>
-          </tbody>
-        </table>
-      </div>
-    </div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
deleted file mode 100644
index 951cc1c..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
+++ /dev/null
@@ -1,40 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<table class="table table-properties">
-  <thead>
-    <tr>
-      <th colspan="2">Some statistics</th>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Operator</td>
-      <td>1</td>
-    </tr>
-    <tr>
-      <td>Parallelism</td>
-      <td>2</td>
-    </tr>
-    <tr>
-      <td>Subtasks-per-instance</td>
-      <td>3</td>
-    </tr>
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
deleted file mode 100644
index 2f22576..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
+++ /dev/null
@@ -1,23 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div class="canvas-wrapper">
-  <div timeline="timeline" vertices="vertices" jobid="jobid" class="timeline-canvas"></div>
-</div>
-<div ui-view="vertex"></div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
deleted file mode 100644
index 1a4bd06..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
+++ /dev/null
@@ -1,30 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<div ng-if="vertex" class="panel panel-default panel-multi">
-  <div class="panel-heading clearfix">
-    <div class="panel-title">{{ vertex.groupvertex.groupvertexname | humanizeText }}</div>
-  </div>
-  <div class="panel-body">
-    <div class="canvas-wrapper">
-      <div vertex="vertex" data="vertex" class="timeline-canvas"></div>
-    </div>
-    <div id="timeline1"></div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html b/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
deleted file mode 100644
index e175d07..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
+++ /dev/null
@@ -1,53 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Running Jobs</div>
-</nav>
-<div id="content-inner">
-  <table class="table table-hover table-clickable">
-    <thead>
-      <tr>
-        <th>Start Time</th>
-        <th>End Time</th>
-        <th>Duration</th>
-        <th>Job Name</th>
-        <th>Job ID</th>
-        <th>Tasks</th>
-        <th>Status</th>
-      </tr>
-    </thead>
-    <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-        <td>{{job.duration}} ms</td>
-        <td>{{job.name}}</td>
-        <td>{{job.jid}}</td>
-        <td class="label-group">
-          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-        </td>
-        <td> 
-          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-        </td>
-      </tr>
-    </tbody>
-  </table>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/partials/overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/overview.html b/flink-runtime-web/src/main/resources/web/partials/overview.html
deleted file mode 100644
index ec3c580..0000000
--- a/flink-runtime-web/src/main/resources/web/partials/overview.html
+++ /dev/null
@@ -1,147 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
--->
-<nav class="navbar navbar-default navbar-fixed-top navbar-main">
-  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
-  <div class="navbar-title">Overview</div>
-</nav>
-<div id="content-inner">
-  <div class="row">
-    <div class="col-md-6">
-      <div class="panel panel-default panel-dashboard">
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-tasks fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview.taskmanagers}}</div>
-              <div>Task Managers</div>
-            </div>
-          </div>
-        </div>
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-folder fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview["slots-total"]}}</div>
-              <div>Task Slots</div>
-            </div>
-          </div>
-        </div>
-        <div class="panel-heading">
-          <div class="row">
-            <div class="col-xs-3"><i class="fa fa-folder-o fa-3x"></i></div>
-            <div class="col-xs-9 text-right">
-              <div class="huge">{{overview["slots-available"]}}</div>
-              <div>Available Task Slots</div>
-            </div>
-          </div>
-        </div>
-      </div>
-    </div>
-    <div class="col-md-6">
-      <div class="panel panel-default panel-lg">
-        <div class="panel-heading">Total Jobs</div>
-        <div class="list-group">
-          <div class="list-group-item">
-            <div class="badge badge-primary">{{overview["jobs-running"]}}</div>Running
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-success">{{overview["jobs-finished"]}}</div>Finished
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-info">{{overview["jobs-cancelled"]}}</div>Canceled
-          </div>
-          <div class="list-group-item">
-            <div class="badge badge-danger">{{overview["jobs-failed"]}}</div>Failed
-          </div>
-        </div>
-      </div>
-    </div>
-  </div>
-  <div class="panel panel-default">
-    <div class="panel-heading">
-      <h3 class="panel-title">Running Jobs</h3>
-    </div>
-    <div class="panel-body">
-      <table class="table table-hover table-clickable">
-        <thead>
-          <tr>
-            <th>Start Time</th>
-            <th>End Time</th>
-            <th>Duration</th>
-            <th>Job Name</th>
-            <th>Job ID</th>
-            <th>Tasks</th>
-            <th>Status</th>
-          </tr>
-        </thead>
-        <tbody>
-          <tr ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job.duration}} ms</td>
-            <td>{{job.name}}</td>
-            <td>{{job.jid}}</td>
-            <td class="label-group">
-              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-            </td>
-            <td> 
-              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-            </td>
-          </tr>
-        </tbody>
-      </table>
-    </div>
-  </div>
-  <div class="panel panel-default">
-    <div class="panel-heading">
-      <h3 class="panel-title">Completed Jobs</h3>
-    </div>
-    <div class="panel-body">
-      <table class="table table-hover table-clickable">
-        <thead>
-          <tr>
-            <th>Start Time</th>
-            <th>End Time</th>
-            <th>Duration</th>
-            <th>Job Name</th>
-            <th>Job ID</th>
-            <th>Tasks</th>
-            <th>Status</th>
-          </tr>
-        </thead>
-        <tbody>
-          <tr ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
-            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
-            <td>{{job.duration}} ms</td>
-            <td>{{job.name}}</td>
-            <td>{{job.jid}}</td>
-            <td class="label-group">
-              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
-            </td>
-            <td> 
-              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
-            </td>
-          </tr>
-        </tbody>
-      </table>
-    </div>
-  </div>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 5167d13..68b00dc 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -46,6 +46,7 @@ import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Scanner;
@@ -62,7 +63,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 	private final static FiniteDuration TestTimeout = new FiniteDuration(2, TimeUnit.MINUTES);
 
-	private final String MAIN_RESOURCES_PATH = getClass().getResource("/../classes/web").getPath();
+	private final String MAIN_RESOURCES_PATH = getClass().getResource("/web").getPath();
 
 	/**
 	 * Tests operation of the monitor in standalone operation.
@@ -82,10 +83,13 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			ActorSystem jmActorSystem = flink.jobManagerActorSystems().get().head();
 			ActorRef jmActor = flink.jobManagerActors().get().head();
 
+			File logDir = temporaryFolder.newFolder("log");
+			Files.createFile(new File(logDir, "jobmanager.log").toPath());
+			Files.createFile(new File(logDir, "jobmanager.out").toPath());
+
 			Configuration monitorConfig = new Configuration();
-			monitorConfig.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
-			monitorConfig.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
 			monitorConfig.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+			monitorConfig.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDir.getAbsolutePath());
 
 			// Needs to match the leader address from the leader retrieval service
 			String jobManagerAddress = AkkaUtils.getAkkaURL(jmActorSystem, jmActor);
@@ -143,9 +147,13 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			final Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
 				zooKeeper.getConnectString(),
 				temporaryFolder.getRoot().getPath());
-			config.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
-			config.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
+
+			File logDir = temporaryFolder.newFolder();
+			Files.createFile(new File(logDir, "jobmanager.log").toPath());
+			Files.createFile(new File(logDir, "jobmanager.out").toPath());
+
 			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDir.getAbsolutePath());
 
 			for (int i = 0; i < jobManagerSystem.length; i++) {
 				jobManagerSystem[i] = AkkaUtils.createActorSystem(new Configuration(),
@@ -280,10 +288,13 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 		try (TestingServer zooKeeper = new TestingServer()) {
 
+			File logDir = temporaryFolder.newFolder();
+			Files.createFile(new File(logDir, "jobmanager.log").toPath());
+			Files.createFile(new File(logDir, "jobmanager.out").toPath());
+
 			final Configuration config = new Configuration();
-			config.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
-			config.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
 			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDir.getAbsolutePath());
 			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
 			config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/gulpfile.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/gulpfile.js b/flink-runtime-web/web-dashboard/gulpfile.js
index 2abfcce..737519e 100644
--- a/flink-runtime-web/web-dashboard/gulpfile.js
+++ b/flink-runtime-web/web-dashboard/gulpfile.js
@@ -43,7 +43,7 @@ var path = require('path');
 var environment = 'development';
 var paths = {
   src: './app/',
-  dest: '../src/main/resources/web/',
+  dest: './web/',
   vendor: './bower_components/',
   vendorLocal: './vendor-local/',
   assets: './assets/',
@@ -168,7 +168,7 @@ gulp.task('watch', function () {
 });
 
 gulp.task('serve', serve({
-  root: '../src/main/resources/web/',
+  root: 'web',
   port: 3001
 }));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/css/index.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/css/index.css b/flink-runtime-web/web-dashboard/web/css/index.css
new file mode 100644
index 0000000..43f7d98
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/css/index.css
@@ -0,0 +1,596 @@
+/*
+ * 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.
+ */
+#sidebar {
+  overflow: hidden;
+  position: fixed;
+  left: -250px;
+  top: 0;
+  bottom: 0;
+  height: 100%;
+  width: 250px;
+  background: #151515;
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+  -webkit-box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
+  box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
+}
+#sidebar.sidebar-visible {
+  left: 0;
+}
+#sidebar .logo {
+  width: auto;
+  height: 22px;
+}
+#sidebar .logo img {
+  display: inline-block;
+}
+#sidebar .navbar-static-top {
+  overflow: hidden;
+  height: 51px;
+}
+#sidebar .navbar-static-top .navbar-header {
+  width: 100%;
+}
+#sidebar .navbar-brand.navbar-brand-text {
+  font-size: 14px;
+  font-weight: bold;
+  color: #fff;
+  padding-left: 0;
+}
+#sidebar .nav > li > a {
+  color: #aaa;
+  margin-bottom: 1px;
+}
+#sidebar .nav > li > a:hover,
+#sidebar .nav > li > a:focus {
+  background-color: rgba(40,40,40,0.5);
+}
+#sidebar .nav > li > a.active {
+  background-color: rgba(100,100,100,0.5);
+}
+#content {
+  background-color: #fff;
+  overflow: hidden;
+  margin-left: 0;
+  padding-top: 70px;
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+}
+#content .navbar-main,
+#content .navbar-main-additional {
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+}
+#content .navbar-main-additional {
+  margin-top: 51px;
+  border-bottom: none;
+  padding: 0 20px;
+}
+#content .navbar-main-additional .nav-tabs {
+  margin: 0 -20px;
+  padding: 0 20px;
+}
+#content .navbar-secondary-additional {
+  border: none;
+  padding: 0 20px;
+  margin-bottom: 0;
+}
+#content .navbar-secondary-additional .nav-tabs {
+  margin: 0 -20px;
+}
+#content.sidebar-visible {
+  margin-left: 250px;
+}
+#content.sidebar-visible .navbar-main,
+#content.sidebar-visible .navbar-main-additional {
+  left: 250px;
+}
+#content #fold-button {
+  display: inline-block;
+  margin-left: 20px;
+}
+#content #content-inner {
+  padding: 0px 20px 20px 20px;
+}
+#content #content-inner.has-navbar-main-additional {
+  padding-top: 42px;
+}
+.page-header {
+  margin: 0 0 20px 0;
+}
+.nav > li > a,
+.nav > li > a:hover,
+.nav > li > a:focus {
+  color: #aaa;
+  background-color: transparent;
+  border-bottom: 2px solid transparent;
+}
+.nav > li.active > a,
+.nav > li.active > a:hover,
+.nav > li.active > a:focus {
+  color: #000;
+  border-bottom: 2px solid #000;
+}
+.nav.nav-tabs {
+  margin-bottom: 20px;
+}
+.table .table {
+  background-color: transparent;
+}
+.table th {
+  font-weight: normal;
+  color: #999;
+}
+.table td.td-long {
+  width: 20%;
+  white-space: pre-wrap;
+  white-space: -moz-pre-wrap;
+  white-space: -pre-wrap;
+  white-space: -o-pre-wrap;
+  word-wrap: break-word;
+}
+.table.table-clickable tr {
+  cursor: pointer;
+}
+.table.table-inner {
+  background-color: transparent;
+}
+.table.table-properties {
+  table-layout: fixed;
+  white-space: nowrap;
+}
+.table.table-properties td {
+  width: 50%;
+  white-space: nowrap;
+  overflow: hidden;
+  -o-text-overflow: ellipsis;
+  text-overflow: ellipsis;
+}
+.table.table-body-hover > tbody {
+  border-top: none;
+  border-left: 2px solid transparent;
+}
+.table.table-body-hover > tbody.active {
+  border-left: 2px solid #000;
+}
+.table.table-body-hover > tbody:hover td:not(.tab-column),
+.table.table-body-hover > tbody.active td:not(.tab-column) {
+  background-color: #f0f0f0;
+}
+.table.table-body-hover > tbody:hover td.tab-column li.active,
+.table.table-body-hover > tbody.active td.tab-column li.active {
+  background-color: #f0f0f0;
+}
+.table.table-activable th.tab-column,
+.table.table-activable td.tab-column {
+  border-top: none;
+  width: 47px;
+}
+.table.table-activable td.tab-column {
+  border-right: 1px solid #ddd;
+}
+.table.table-activable td {
+  position: relative;
+}
+.table .small-label {
+  text-transform: uppercase;
+  font-size: 13px;
+  color: #999;
+}
+.panel.panel-dashboard .huge {
+  font-size: 28px;
+}
+.panel.panel-lg {
+  font-size: 16px;
+}
+.panel.panel-lg .badge {
+  font-size: 14px;
+}
+.navbar-secondary {
+  overflow: auto;
+}
+.navbar-main .navbar-title,
+.navbar-secondary .navbar-title,
+.navbar-main-additional .navbar-title,
+.panel.panel-multi .navbar-title,
+.navbar-secondary-additional .navbar-title,
+.navbar-main .panel-title,
+.navbar-secondary .panel-title,
+.navbar-main-additional .panel-title,
+.panel.panel-multi .panel-title,
+.navbar-secondary-additional .panel-title {
+  float: left;
+  font-size: 18px;
+  padding: 12px 20px 13px 10px;
+  color: #333;
+  display: inline-block;
+}
+.navbar-main .navbar-info,
+.navbar-secondary .navbar-info,
+.navbar-main-additional .navbar-info,
+.panel.panel-multi .navbar-info,
+.navbar-secondary-additional .navbar-info,
+.navbar-main .panel-info,
+.navbar-secondary .panel-info,
+.navbar-main-additional .panel-info,
+.panel.panel-multi .panel-info,
+.navbar-secondary-additional .panel-info {
+  float: left;
+  font-size: 14px;
+  padding: 15px 15px 15px 15px;
+  color: #999;
+  display: inline-block;
+  border-right: 1px solid #e7e7e7;
+  overflow: hidden;
+}
+.navbar-main .navbar-info .overflow,
+.navbar-secondary .navbar-info .overflow,
+.navbar-main-additional .navbar-info .overflow,
+.panel.panel-multi .navbar-info .overflow,
+.navbar-secondary-additional .navbar-info .overflow,
+.navbar-main .panel-info .overflow,
+.navbar-secondary .panel-info .overflow,
+.navbar-main-additional .panel-info .overflow,
+.panel.panel-multi .panel-info .overflow,
+.navbar-secondary-additional .panel-info .overflow {
+  position: absolute;
+  display: block;
+  -o-text-overflow: ellipsis;
+  text-overflow: ellipsis;
+  overflow: hidden;
+  height: 22px;
+  line-height: 22px;
+  vertical-align: middle;
+}
+.navbar-main .navbar-info.first,
+.navbar-secondary .navbar-info.first,
+.navbar-main-additional .navbar-info.first,
+.panel.panel-multi .navbar-info.first,
+.navbar-secondary-additional .navbar-info.first,
+.navbar-main .panel-info.first,
+.navbar-secondary .panel-info.first,
+.navbar-main-additional .panel-info.first,
+.panel.panel-multi .panel-info.first,
+.navbar-secondary-additional .panel-info.first {
+  border-left: 1px solid #e7e7e7;
+}
+.navbar-main .navbar-info.last,
+.navbar-secondary .navbar-info.last,
+.navbar-main-additional .navbar-info.last,
+.panel.panel-multi .navbar-info.last,
+.navbar-secondary-additional .navbar-info.last,
+.navbar-main .panel-info.last,
+.navbar-secondary .panel-info.last,
+.navbar-main-additional .panel-info.last,
+.panel.panel-multi .panel-info.last,
+.navbar-secondary-additional .panel-info.last {
+  border-right: none;
+}
+.panel.panel-multi .panel-heading {
+  padding: 0;
+}
+.panel.panel-multi .panel-heading .panel-info.thin {
+  padding: 8px 10px;
+}
+.panel.panel-multi .panel-body {
+  padding: 10px;
+  background-color: #fdfdfd;
+  color: #999;
+  font-size: 13px;
+}
+.panel.panel-multi .panel-body.clean {
+  color: inherit;
+  font-size: inherit;
+}
+.navbar-main-additional,
+.navbar-secondary-additional {
+  min-height: 40px;
+  background-color: #fdfdfd;
+}
+.navbar-main-additional .navbar-info,
+.navbar-secondary-additional .navbar-info {
+  font-size: 13px;
+  padding: 10px 15px 10px 15px;
+}
+.nav-top-affix.affix {
+  width: 100%;
+  top: 50px;
+  margin-left: -20px;
+  padding-left: 20px;
+  margin-right: -20px;
+  padding-right: 20px;
+  background-color: #fff;
+  z-index: 1;
+}
+.badge-default[href]:hover,
+.badge-default[href]:focus {
+  background-color: #808080;
+}
+.badge-primary {
+  background-color: #428bca;
+}
+.badge-primary[href]:hover,
+.badge-primary[href]:focus {
+  background-color: #3071a9;
+}
+.badge-success {
+  background-color: #5cb85c;
+}
+.badge-success[href]:hover,
+.badge-success[href]:focus {
+  background-color: #449d44;
+}
+.badge-info {
+  background-color: #5bc0de;
+}
+.badge-info[href]:hover,
+.badge-info[href]:focus {
+  background-color: #31b0d5;
+}
+.badge-warning {
+  background-color: #f0ad4e;
+}
+.badge-warning[href]:hover,
+.badge-warning[href]:focus {
+  background-color: #ec971f;
+}
+.badge-danger {
+  background-color: #d9534f;
+}
+.badge-danger[href]:hover,
+.badge-danger[href]:focus {
+  background-color: #c9302c;
+}
+.indicator {
+  display: inline-block;
+  margin-right: 15px;
+}
+.indicator.indicator-primary {
+  color: #428bca;
+}
+.indicator.indicator-success {
+  color: #5cb85c;
+}
+.indicator.indicator-info {
+  color: #5bc0de;
+}
+.indicator.indicator-warning {
+  color: #f0ad4e;
+}
+.indicator.indicator-danger {
+  color: #d9534f;
+}
+pre.exception {
+  border: none;
+  background-color: transparent;
+  padding: 0;
+  margin: 0;
+}
+.nav-tabs.tabs-vertical {
+  position: absolute;
+  left: 0;
+  top: 0;
+  border-bottom: none;
+  z-index: 100;
+}
+.nav-tabs.tabs-vertical li {
+  float: none;
+  margin-bottom: 0;
+  margin-right: -1px;
+}
+.nav-tabs.tabs-vertical li > a {
+  margin-right: 0;
+  -webkit-border-radius: 0;
+  border-radius: 0;
+  border-bottom: none;
+  border-left: 2px solid transparent;
+}
+.nav-tabs.tabs-vertical li > a:hover,
+.nav-tabs.tabs-vertical li > a:focus {
+  border-bottom: none;
+  border-left: 2px solid #000;
+}
+.nav-tabs.tabs-vertical li.active > a {
+  border-bottom: none;
+  border-left: 2px solid #000;
+}
+.navbar-main .navbar-title,
+.navbar-secondary .navbar-title,
+.navbar-main-additional .navbar-title,
+.navbar-secondary-additional .navbar-title {
+  padding: 12px 20px 13px 20px;
+}
+livechart {
+  width: 30%;
+  height: 30%;
+  text-align: center;
+}
+.canvas-wrapper {
+  border: 1px solid #ddd;
+  position: relative;
+  margin-bottom: 20px;
+}
+.canvas-wrapper .main-canvas {
+  height: 400px;
+  overflow: hidden;
+}
+.canvas-wrapper .main-canvas .zoom-buttons {
+  position: absolute;
+  top: 10px;
+  right: 10px;
+}
+.label-group .label {
+  display: inline-block;
+  width: 2em;
+  padding-left: 0.1em;
+  padding-right: 0.1em;
+  margin: 0;
+  border-right: 1px solid #fff;
+  -webkit-border-radius: 0;
+  border-radius: 0;
+}
+.label-group .label.label-black {
+  background-color: #000;
+}
+svg.graph {
+  overflow: hidden;
+}
+svg.graph g.type-TK > rect {
+  fill: #00ffd0;
+}
+svg.graph text {
+  font-weight: 300;
+  font-size: 14px;
+}
+svg.graph .node {
+  cursor: pointer;
+}
+svg.graph .node > rect {
+  stroke: #999;
+  stroke-width: 5px;
+  fill: #fff;
+  margin: 0;
+  padding: 0;
+}
+svg.graph .node[active] > rect {
+  fill: #eee;
+}
+svg.graph .node.node-mirror > rect {
+  stroke: #a8a8a8;
+}
+svg.graph .node.node-iteration > rect {
+  stroke: #cd3333;
+}
+svg.graph .node.node-source > rect {
+  stroke: #4ce199;
+}
+svg.graph .node.node-sink > rect {
+  stroke: #e6ec8b;
+}
+svg.graph .node.node-normal > rect {
+  stroke: #3fb6d8;
+}
+svg.graph .node h4 {
+  color: #000;
+}
+svg.graph .node h5 {
+  color: #999;
+}
+svg.graph .edgeLabel rect {
+  fill: #fff;
+}
+svg.graph .edgePath path {
+  stroke: #333;
+  stroke-width: 2px;
+  fill: #333;
+}
+svg.graph .label {
+  color: #777;
+  margin: 0;
+}
+svg.graph .edge-label {
+  font-size: 14px;
+}
+svg.graph .node-label {
+  display: block;
+  margin: 0;
+  text-decoration: none;
+}
+.timeline {
+  overflow: hidden;
+}
+.timeline-canvas {
+  overflow: hidden;
+  padding: 10px;
+}
+.timeline-canvas .bar-container {
+  overflow: hidden;
+}
+.timeline-canvas .timeline-insidelabel,
+.timeline-canvas .timeline-series {
+  cursor: pointer;
+}
+.timeline-canvas.secondary .timeline-insidelabel,
+.timeline-canvas.secondary .timeline-series {
+  cursor: auto;
+}
+.qtip-timeline-bar {
+  font-size: 14px;
+  line-height: 1.4;
+}
+@media (min-width: 1024px) and (max-width: 1279px) {
+  #sidebar {
+    left: 0;
+    width: 160px;
+  }
+  #sidebar .navbar-static-top .navbar-brand-text {
+    display: none;
+  }
+  #content {
+    margin-left: 160px;
+  }
+  #content #fold-button {
+    display: none;
+  }
+  #content .navbar-main,
+  #content .navbar-main-additional {
+    left: 160px;
+  }
+  .table td.td-long {
+    width: 20%;
+  }
+}
+@media (min-width: 1280px) {
+  #sidebar {
+    left: 0;
+  }
+  #content {
+    margin-left: 250px;
+  }
+  #content #fold-button {
+    display: none;
+  }
+  #content .navbar-main,
+  #content .navbar-main-additional {
+    left: 250px;
+  }
+  .table td.td-long {
+    width: 30%;
+  }
+}
+#total-mem {
+  background-color: #7cb5ec;
+}
+#heap-mem {
+  background-color: #434348;
+}
+#non-heap-mem {
+  background-color: #90ed7d;
+}
+a.show-pointer {
+  cursor: pointer;
+}


[17/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
new file mode 100644
index 0000000..da6b75b
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobmanager/config.html
@@ -0,0 +1,33 @@
+
+<!--
+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.
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th>Key</th>
+      <th>Value</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="entry in jobmanager.config | orderBy: 'key'">
+      <td>{{entry.key}}</td>
+      <td>{{entry.value}}</td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
new file mode 100644
index 0000000..02c2f47
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobmanager/index.html
@@ -0,0 +1,33 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Job Manager</div>
+</nav>
+<nav class="navbar navbar-default navbar-fixed-top navbar-main-additional">
+  <ul class="nav nav-tabs">
+    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
+    <li ui-sref-active="active"><a ui-sref=".log">Logs</a></li>
+    <li ui-sref-active="active"><a ui-sref=".stdout">Stdout</a></li>
+  </ul>
+</nav>
+<div id="content-inner" class="has-navbar-main-additional">
+  <div ui-view="details"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html b/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
new file mode 100644
index 0000000..df6a817
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobmanager/stdout.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">
+        <div class="row">
+          <div class="col-xs-10">Job Manager Output</div>
+          <div class="col-xs-1 text-right"><a ng-click="reloadData()" class="show-pointer"><i class="fa fa-refresh"></i></a></div>
+          <div class="col-xs-1 text-left"><a href="jobmanager/stdout"><i class="fa fa-download"></i></a></div>
+        </div>
+      </th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td colspan="2">
+        <pre>{{jobmanager.stdout}}</pre>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html b/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
new file mode 100644
index 0000000..b76278d
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/completed-jobs.html
@@ -0,0 +1,53 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Completed Jobs</div>
+</nav>
+<div id="content-inner">
+  <table class="table table-hover table-clickable">
+    <thead>
+      <tr>
+        <th>Start Time</th>
+        <th>End Time</th>
+        <th>Duration</th>
+        <th>Job Name</th>
+        <th>Job ID</th>
+        <th>Tasks</th>
+        <th>Status</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job.duration}} ms</td>
+        <td>{{job.name}}</td>
+        <td>{{job.jid}}</td>
+        <td class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+        </td>
+        <td> 
+          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+        </td>
+      </tr>
+    </tbody>
+  </table>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
new file mode 100644
index 0000000..a7a5d9d
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.config.html
@@ -0,0 +1,57 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table ng-if="job['execution-config']" class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">Execution configuration</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Execution mode</td>
+      <td>{{ job['execution-config']['execution-mode'] }}</td>
+    </tr>
+    <tr>
+      <td>Max. number of execution retries</td>
+      <td>{{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }}</td>
+    </tr>
+    <tr>
+      <td>Job parallelism</td>
+      <td>{{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }}</td>
+    </tr>
+    <tr>
+      <td>Object reuse mode</td>
+      <td>{{ job['execution-config']['object-reuse-mode'] }}</td>
+    </tr>
+  </tbody>
+</table>
+<table ng-if="job['execution-config']['user-config']" class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">User configuration</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="property in job['execution-config']['user-config']">
+      <td>{{property.name}}</td>
+      <td table-property="table-property" value="property.value"></td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
new file mode 100644
index 0000000..a5f6676
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.exceptions.html
@@ -0,0 +1,38 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div ng-if="exceptions['root-exception']" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">Root exception</div>
+  </div>
+  <div class="panel-body">
+    <pre class="exception">{{ exceptions['root-exception'] }}</pre>
+  </div>
+</div>
+<div ng-repeat="exception in exceptions['all-exceptions']" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <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>
+  <div class="panel-body">
+    <pre class="exception">{{ exception.exception }}</pre>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
new file mode 100644
index 0000000..9d3e171
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.html
@@ -0,0 +1,48 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">
+    <indicator-primary status="{{job.state}}"></indicator-primary>{{ job.name }}
+  </div>
+  <div class="navbar-info first last hidden-xs hidden-sm">{{ job.jid }}</div>
+  <div class="navbar-info first last">
+    <div class="label-group">
+      <bs-label status="{{status}}" ng-repeat="(status, value) in job['status-counts']">{{value}}</bs-label>
+    </div>
+  </div>
+  <div class="navbar-info first last hidden-xs hidden-sm">{{ job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}<span ng-if="job['end-time'] &gt; -1">
+      - 
+      {{ job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></div>
+  <div ng-if="job.duration &gt; -1" class="navbar-info last first">{{job.duration}} ms</div>
+</nav>
+<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
+  <ul class="nav nav-tabs">
+    <li ui-sref-active="active"><a ui-sref=".plan.overview">Plan</a></li>
+    <li ui-sref-active="active"><a ui-sref=".statistics">Job Accumulators / Statistics</a></li>
+    <li ui-sref-active="active"><a ui-sref=".timeline">Timeline</a></li>
+    <li ui-sref-active="active"><a ui-sref=".exceptions">Exceptions</a></li>
+    <li ui-sref-active="active"><a ui-sref=".properties">Properties</a></li>
+    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
+  </ul>
+</nav>
+<div id="content-inner" class="has-navbar-main-additional">
+  <div ui-view="details"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
new file mode 100644
index 0000000..f2c4143
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.html
@@ -0,0 +1,31 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div class="canvas-wrapper">
+  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
+</div>
+<div ng-if="plan" class="panel panel-default panel-multi">
+  <nav class="navbar navbar-default navbar-secondary-additional">
+    <ul class="nav nav-tabs">
+      <li ui-sref-active="active"><a ui-sref=".overview({nodeid: nodeid})">Overview</a></li>
+      <li ui-sref-active="active"><a ui-sref=".accumulators({nodeid: nodeid})">Accumulators</a></li>
+    </ul>
+  </nav>
+  <div ui-view="node-details" class="panel-body clean"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
new file mode 100644
index 0000000..8de3921
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.accumulators.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table class="table table-body-hover table-clickable table-activable">
+  <thead>
+    <tr>
+      <th>Name</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="v.id == nodeid || changeNode(v.id)">
+    <tr ng-if="v.type == 'regular'">
+      <td>{{ v.name | humanizeText }}</td>
+      <td> 
+        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
+      </td>
+    </tr>
+    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
+      <td colspan="10">
+        <div ng-include=" 'partials/jobs/job.plan.node.accumulators.html' "></div>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
new file mode 100644
index 0000000..1706d3e
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node-list.overview.html
@@ -0,0 +1,60 @@
+
+<!--
+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.
+
+-->
+<table class="table table-body-hover table-clickable table-activable">
+  <thead>
+    <tr>
+      <th>Start Time</th>
+      <th>End Time</th>
+      <th>Duration</th>
+      <th>Name</th>
+      <th>Bytes read</th>
+      <th>Records read</th>
+      <th>Bytes written</th>
+      <th>Records written</th>
+      <th>Tasks</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="changeNode(v.id)">
+    <tr ng-if="v.type == 'regular'">
+      <td><span ng-if="v['start-time'] &gt; -1">{{ v['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="v['end-time'] &gt; -1">{{ v['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="v.duration &gt; -1">{{ v.duration }} ms</span></td>
+      <td class="td-long">{{ v.name | humanizeText }}</td>
+      <td>{{ v.metrics['read-bytes'] }}</td>
+      <td>{{ v.metrics['read-records'] }}</td>
+      <td>{{ v.metrics['write-bytes'] }}</td>
+      <td>{{ v.metrics['write-records'] }}</td>
+      <td>
+        <div class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(index, status) in stateList">{{v.tasks[status]}}</bs-label>
+        </div>
+      </td>
+      <td> 
+        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
+      </td>
+    </tr>
+    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
+      <td colspan="10">
+        <div ng-include=" 'partials/jobs/job.plan.node.subtasks.html' "></div>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
new file mode 100644
index 0000000..e7dcf2c
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.accumulators.html
@@ -0,0 +1,68 @@
+
+<!--
+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.
+
+-->
+<div ng-if="accumulators.length == 0">
+  <p><i>No accumulators</i></p>
+</div>
+<div ng-if="accumulators &amp;&amp; accumulators.length &gt; 0">
+  <table class="table table-hover table-clickable table-activable table-inner">
+    <thead>
+      <tr>
+        <th>Name</th>
+        <th>Type</th>
+        <th>Value</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="accumulator in accumulators">
+        <td width="30%">{{ accumulator.name }}</td>
+        <td width="30%">{{ accumulator.type }}</td>
+        <td width="30%">{{ accumulator.value }}</td>
+      </tr>
+    </tbody>
+  </table>
+  <div ng-if="!nodeUnfolded"><a ng-click="toggleFold()" class="btn btn-default">
+      Show subtasks
+       <i class="fa fa-chevron-down"></i></a><a ng-click="deactivateNode(); $event.stopPropagation()" title="Fold" class="btn btn-default pull-right"><i class="fa fa-chevron-up"></i></a></div>
+  <div ng-if="nodeUnfolded &amp;&amp; subtaskAccumulators &amp;&amp; subtaskAccumulators.length &gt; 0"><a ng-click="toggleFold()" class="btn btn-default">
+      Hide subtasks
+       <i class="fa fa-chevron-up"></i></a>
+    <table class="table table-hover table-clickable table-activable table-inner">
+      <thead>
+        <tr>
+          <th>Name</th>
+          <th>Type</th>
+          <th>Value</th>
+        </tr>
+      </thead>
+      <tbody ng-if="subtask['user-accumulators'] &amp;&amp; subtask['user-accumulators'].length &gt; 0" ng-repeat="subtask in subtaskAccumulators">
+        <tr>
+          <td colwidth="3">
+            <div class="small-label">({{ subtask.subtask }}) {{ subtask.host }}, attempt: {{ subtask.attempt + 1 }}</div>
+          </td>
+        </tr>
+        <tr ng-repeat="accumulator in subtask['user-accumulators']">
+          <td width="30%">{{ accumulator.name }}</td>
+          <td width="30%">{{ accumulator.type }}</td>
+          <td width="30%">{{ accumulator.value }}</td>
+        </tr>
+      </tbody>
+    </table>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
new file mode 100644
index 0000000..40b16bc
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.plan.node.subtasks.html
@@ -0,0 +1,52 @@
+
+<!--
+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.
+
+
+-->
+<table ng-if="subtasks" class="table table-hover table-clickable table-activable table-inner">
+  <thead>
+    <tr>
+      <th>Start Time</th>
+      <th>End Time</th>
+      <th>Duration</th>
+      <th>Bytes read</th>
+      <th>Records read</th>
+      <th>Bytes written</th>
+      <th>Records written</th>
+      <th>Attempt</th>
+      <th>Host</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="subtask in subtasks">
+      <td><span ng-if="subtask['start-time'] &gt; -1">{{ subtask['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="subtask['end-time'] &gt; -1">{{ subtask['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="subtask.duration &gt; -1">{{ subtask.duration }} ms</span></td>
+      <td><span ng-if="subtask.metrics['read-bytes'] &gt; -1">{{ subtask.metrics['read-bytes'] }}</span></td>
+      <td><span ng-if="subtask.metrics['read-records'] &gt; -1">{{ subtask.metrics['read-records'] }}</span></td>
+      <td><span ng-if="subtask.metrics['write-bytes'] &gt; -1">{{ subtask.metrics['write-bytes'] }}</span></td>
+      <td><span ng-if="subtask.metrics['write-records'] &gt; -1">{{ subtask.metrics['write-records'] }}</span></td>
+      <td>{{ subtask.attempt + 1 }}</td>
+      <td>{{ subtask.host }}</td>
+      <td> 
+        <bs-label status="{{subtask.status}}">{{subtask.status}}</bs-label>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
new file mode 100644
index 0000000..907afd3
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.properties.html
@@ -0,0 +1,140 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div class="canvas-wrapper">
+  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
+</div>
+<div ng-if="node" class="panel panel-default">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">{{ node.description | humanizeText }}</div>
+  </div>
+  <div class="panel-body clean">
+    <div class="row">
+      <div class="col-sm-6 col-md-4">
+        <table ng-if="node.optimizer_properties.global_properties" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Global Data Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.global_properties">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+        <table ng-if="node.optimizer_properties.local_properties" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Local Data Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.local_properties">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+        <div class="visible-xs visible-sm">
+          <table class="table table-properties">
+            <thead>
+              <tr>
+                <th colspan="2">Properties</th>
+              </tr>
+            </thead>
+            <tbody>
+              <tr>
+                <td>Operator</td>
+                <td table-property="table-property" value="node.operator_strategy"></td>
+              </tr>
+              <tr>
+                <td>Parallelism</td>
+                <td table-property="table-property" value="node.parallelism"></td>
+              </tr>
+            </tbody>
+          </table>
+        </div>
+      </div>
+      <div class="hidden-sm col-md-4">
+        <table class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr>
+              <td>Operator</td>
+              <td table-property="table-property" value="node.operator_strategy"></td>
+            </tr>
+            <tr>
+              <td>Parallelism</td>
+              <td table-property="table-property" value="node.parallelism"></td>
+            </tr>
+          </tbody>
+        </table>
+        <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Size Estimates</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.estimates">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+      </div>
+      <div class="col-sm-6 col-md-4">
+        <div class="visible-xs visible-sm">
+          <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
+            <thead>
+              <tr>
+                <th colspan="2">Size Estimates</th>
+              </tr>
+            </thead>
+            <tbody>
+              <tr ng-repeat="property in node.optimizer_properties.estimates">
+                <td>{{property.name}}</td>
+                <td table-property="table-property" value="property.value"></td>
+              </tr>
+            </tbody>
+          </table>
+        </div>
+        <table ng-if="node.optimizer_properties.costs" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Cost Estimates</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.costs">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+      </div>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
new file mode 100644
index 0000000..951cc1c
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.statistics.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">Some statistics</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Operator</td>
+      <td>1</td>
+    </tr>
+    <tr>
+      <td>Parallelism</td>
+      <td>2</td>
+    </tr>
+    <tr>
+      <td>Subtasks-per-instance</td>
+      <td>3</td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
new file mode 100644
index 0000000..2f22576
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.html
@@ -0,0 +1,23 @@
+
+<!--
+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.
+
+-->
+<div class="canvas-wrapper">
+  <div timeline="timeline" vertices="vertices" jobid="jobid" class="timeline-canvas"></div>
+</div>
+<div ui-view="vertex"></div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
new file mode 100644
index 0000000..1a4bd06
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/job.timeline.vertex.html
@@ -0,0 +1,30 @@
+
+<!--
+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.
+
+-->
+<div ng-if="vertex" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">{{ vertex.groupvertex.groupvertexname | humanizeText }}</div>
+  </div>
+  <div class="panel-body">
+    <div class="canvas-wrapper">
+      <div vertex="vertex" data="vertex" class="timeline-canvas"></div>
+    </div>
+    <div id="timeline1"></div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html b/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
new file mode 100644
index 0000000..e175d07
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/jobs/running-jobs.html
@@ -0,0 +1,53 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Running Jobs</div>
+</nav>
+<div id="content-inner">
+  <table class="table table-hover table-clickable">
+    <thead>
+      <tr>
+        <th>Start Time</th>
+        <th>End Time</th>
+        <th>Duration</th>
+        <th>Job Name</th>
+        <th>Job ID</th>
+        <th>Tasks</th>
+        <th>Status</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job.duration}} ms</td>
+        <td>{{job.name}}</td>
+        <td>{{job.jid}}</td>
+        <td class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+        </td>
+        <td> 
+          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+        </td>
+      </tr>
+    </tbody>
+  </table>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/partials/overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/partials/overview.html b/flink-runtime-web/src/main/resources/web/partials/overview.html
new file mode 100644
index 0000000..ec3c580
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/partials/overview.html
@@ -0,0 +1,147 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Overview</div>
+</nav>
+<div id="content-inner">
+  <div class="row">
+    <div class="col-md-6">
+      <div class="panel panel-default panel-dashboard">
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-tasks fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview.taskmanagers}}</div>
+              <div>Task Managers</div>
+            </div>
+          </div>
+        </div>
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-folder fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview["slots-total"]}}</div>
+              <div>Task Slots</div>
+            </div>
+          </div>
+        </div>
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-folder-o fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview["slots-available"]}}</div>
+              <div>Available Task Slots</div>
+            </div>
+          </div>
+        </div>
+      </div>
+    </div>
+    <div class="col-md-6">
+      <div class="panel panel-default panel-lg">
+        <div class="panel-heading">Total Jobs</div>
+        <div class="list-group">
+          <div class="list-group-item">
+            <div class="badge badge-primary">{{overview["jobs-running"]}}</div>Running
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-success">{{overview["jobs-finished"]}}</div>Finished
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-info">{{overview["jobs-cancelled"]}}</div>Canceled
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-danger">{{overview["jobs-failed"]}}</div>Failed
+          </div>
+        </div>
+      </div>
+    </div>
+  </div>
+  <div class="panel panel-default">
+    <div class="panel-heading">
+      <h3 class="panel-title">Running Jobs</h3>
+    </div>
+    <div class="panel-body">
+      <table class="table table-hover table-clickable">
+        <thead>
+          <tr>
+            <th>Start Time</th>
+            <th>End Time</th>
+            <th>Duration</th>
+            <th>Job Name</th>
+            <th>Job ID</th>
+            <th>Tasks</th>
+            <th>Status</th>
+          </tr>
+        </thead>
+        <tbody>
+          <tr ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job.duration}} ms</td>
+            <td>{{job.name}}</td>
+            <td>{{job.jid}}</td>
+            <td class="label-group">
+              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+            </td>
+            <td> 
+              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+            </td>
+          </tr>
+        </tbody>
+      </table>
+    </div>
+  </div>
+  <div class="panel panel-default">
+    <div class="panel-heading">
+      <h3 class="panel-title">Completed Jobs</h3>
+    </div>
+    <div class="panel-body">
+      <table class="table table-hover table-clickable">
+        <thead>
+          <tr>
+            <th>Start Time</th>
+            <th>End Time</th>
+            <th>Duration</th>
+            <th>Job Name</th>
+            <th>Job ID</th>
+            <th>Tasks</th>
+            <th>Status</th>
+          </tr>
+        </thead>
+        <tbody>
+          <tr ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job.duration}} ms</td>
+            <td>{{job.name}}</td>
+            <td>{{job.jid}}</td>
+            <td class="label-group">
+              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+            </td>
+            <td> 
+              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+            </td>
+          </tr>
+        </tbody>
+      </table>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
new file mode 100644
index 0000000..26f66b0
--- /dev/null
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.webmonitor;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.webmonitor.files.MimeTypes;
+import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
+import org.junit.Test;
+import org.powermock.reflect.Whitebox;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Scanner;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class WebRuntimeMonitorITCase {
+
+	private final static FiniteDuration TestTimeout = new FiniteDuration(2, TimeUnit.MINUTES);
+
+	private final String MAIN_RESOURCES_PATH = getClass().getResource("/../classes/web").getPath();
+
+	/**
+	 * Tests operation of the monitor in standalone operation.
+	 */
+	@Test
+	public void testStandaloneWebRuntimeMonitor() throws Exception {
+		final Deadline deadline = TestTimeout.fromNow();
+
+		TestingCluster flink = null;
+		WebRuntimeMonitor webMonitor = null;
+
+		try {
+			// Flink w/o a web monitor
+			flink = new TestingCluster(new Configuration());
+			flink.start(true);
+
+			ActorSystem jmActorSystem = flink.jobManagerActorSystems().get().head();
+			ActorRef jmActor = flink.jobManagerActors().get().head();
+
+			Configuration monitorConfig = new Configuration();
+			monitorConfig.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
+			monitorConfig.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
+
+			// Needs to match the leader address from the leader retrieval service
+			String jobManagerAddress = AkkaUtils.getAkkaURL(jmActorSystem, jmActor);
+
+			webMonitor = new WebRuntimeMonitor(monitorConfig, flink.createLeaderRetrievalService(),
+					jmActorSystem);
+
+			webMonitor.start(jobManagerAddress);
+
+			try (HttpTestClient client = new HttpTestClient("localhost", webMonitor.getServerPort())) {
+				String expected = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
+						.useDelimiter("\\A").next();
+
+				// Request the file from the web server
+				client.sendGetRequest("index.html", deadline.timeLeft());
+
+				HttpTestClient.SimpleHttpResponse response = client.getNextResponse(deadline.timeLeft());
+
+				assertEquals(HttpResponseStatus.OK, response.getStatus());
+				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("html"));
+				assertEquals(expected, response.getContent());
+
+				// Simple overview request
+				client.sendGetRequest("/overview", deadline.timeLeft());
+
+				response = client.getNextResponse(deadline.timeLeft());
+				assertEquals(HttpResponseStatus.OK, response.getStatus());
+				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+				assertTrue(response.getContent().contains("\"taskmanagers\":1"));
+			}
+		}
+		finally {
+			if (flink != null) {
+				flink.shutdown();
+			}
+
+			if (webMonitor != null) {
+				webMonitor.stop();
+			}
+		}
+	}
+
+	/**
+	 * Tests that the monitor associated with the following job manager redirects to the leader.
+	 */
+	@Test
+	public void testRedirectToLeader() throws Exception {
+		final Deadline deadline = TestTimeout.fromNow();
+
+		ActorSystem[] jobManagerSystem = new ActorSystem[2];
+		WebRuntimeMonitor[] webMonitor = new WebRuntimeMonitor[2];
+		List<LeaderRetrievalService> leaderRetrievalServices = new ArrayList<>();
+
+		try (TestingServer zooKeeper = new TestingServer()) {
+			final Configuration config = new Configuration();
+			config.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
+			config.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
+			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+			config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
+
+			for (int i = 0; i < jobManagerSystem.length; i++) {
+				jobManagerSystem[i] = AkkaUtils.createActorSystem(new Configuration(),
+						new Some<>(new Tuple2<String, Object>("localhost", 0)));
+			}
+
+			for (int i = 0; i < webMonitor.length; i++) {
+				LeaderRetrievalService lrs = ZooKeeperUtils.createLeaderRetrievalService(config);
+				leaderRetrievalServices.add(lrs);
+				webMonitor[i] = new WebRuntimeMonitor(config, lrs, jobManagerSystem[i]);
+			}
+
+			ActorRef[] jobManager = new ActorRef[2];
+			String[] jobManagerAddress = new String[2];
+			for (int i = 0; i < jobManager.length; i++) {
+				Configuration jmConfig = config.clone();
+				jmConfig.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
+						webMonitor[i].getServerPort());
+
+				jobManager[i] = JobManager.startJobManagerActors(
+						jmConfig, jobManagerSystem[i], StreamingMode.STREAMING)._1();
+
+				jobManagerAddress[i] = AkkaUtils.getAkkaURL(jobManagerSystem[i], jobManager[i]);
+				webMonitor[i].start(jobManagerAddress[i]);
+			}
+
+			LeaderRetrievalService lrs = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalServices.add(lrs);
+			TestingListener leaderListener = new TestingListener();
+			lrs.start(leaderListener);
+
+			leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+			String leaderAddress = leaderListener.getAddress();
+
+			int leaderIndex = leaderAddress.equals(jobManagerAddress[0]) ? 0 : 1;
+			int followerIndex = (leaderIndex + 1) % 2;
+
+			ActorSystem leadingSystem = jobManagerSystem[leaderIndex];
+			ActorSystem followerSystem = jobManagerSystem[followerIndex];
+
+			WebMonitor leadingWebMonitor = webMonitor[leaderIndex];
+			WebMonitor followerWebMonitor = webMonitor[followerIndex];
+
+			// For test stability reason we have to wait until we are sure that both leader
+			// listeners have been notified.
+			JobManagerRetriever leadingRetriever = Whitebox
+					.getInternalState(leadingWebMonitor, "retriever");
+
+			JobManagerRetriever followerRetriever = Whitebox
+					.getInternalState(followerWebMonitor, "retriever");
+
+			// Wait for the initial notifications
+			waitForLeaderNotification(leadingSystem, jobManager[leaderIndex], leadingRetriever, deadline);
+			waitForLeaderNotification(leadingSystem, jobManager[leaderIndex], followerRetriever, deadline);
+
+			try (
+					HttpTestClient leaderClient = new HttpTestClient(
+							"localhost", leadingWebMonitor.getServerPort());
+
+					HttpTestClient followingClient = new HttpTestClient(
+							"localhost", followerWebMonitor.getServerPort())) {
+
+				String expected = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
+						.useDelimiter("\\A").next();
+
+				// Request the file from the leaading web server
+				leaderClient.sendGetRequest("index.html", deadline.timeLeft());
+
+				HttpTestClient.SimpleHttpResponse response = leaderClient.getNextResponse(deadline.timeLeft());
+				assertEquals(HttpResponseStatus.OK, response.getStatus());
+				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("html"));
+				assertEquals(expected, response.getContent());
+
+				// Request the file from the following web server
+				followingClient.sendGetRequest("index.html", deadline.timeLeft());
+				response = followingClient.getNextResponse(deadline.timeLeft());
+				assertEquals(HttpResponseStatus.TEMPORARY_REDIRECT, response.getStatus());
+				assertTrue(response.getLocation().contains("" + leadingWebMonitor.getServerPort()));
+
+				// Kill the leader
+				leadingSystem.shutdown();
+
+				// Wait for the notification of the follower
+				waitForLeaderNotification(followerSystem, jobManager[followerIndex], followerRetriever, deadline);
+
+				// Same request to the new leader
+				followingClient.sendGetRequest("index.html", deadline.timeLeft());
+
+				response = followingClient.getNextResponse(deadline.timeLeft());
+				assertEquals(HttpResponseStatus.OK, response.getStatus());
+				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("html"));
+				assertEquals(expected, response.getContent());
+
+				// Simple overview request
+				followingClient.sendGetRequest("/overview", deadline.timeLeft());
+
+				response = followingClient.getNextResponse(deadline.timeLeft());
+				assertEquals(HttpResponseStatus.OK, response.getStatus());
+				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+				assertTrue(response.getContent().contains("\"taskmanagers\":1") ||
+						response.getContent().contains("\"taskmanagers\":0"));
+			}
+		}
+		finally {
+			for (ActorSystem system : jobManagerSystem) {
+				if (system != null) {
+					system.shutdown();
+				}
+			}
+
+			for (WebMonitor monitor : webMonitor) {
+				monitor.stop();
+			}
+
+			for (LeaderRetrievalService lrs : leaderRetrievalServices) {
+				lrs.stop();
+			}
+		}
+	}
+
+	@Test
+	public void testLeaderNotAvailable() throws Exception {
+		final Deadline deadline = TestTimeout.fromNow();
+
+		ActorSystem actorSystem = null;
+		WebRuntimeMonitor webRuntimeMonitor = null;
+
+		try (TestingServer zooKeeper = new TestingServer()) {
+
+			final Configuration config = new Configuration();
+			config.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
+			config.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
+			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+			config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
+
+			actorSystem = AkkaUtils.createDefaultActorSystem();
+
+			LeaderRetrievalService leaderRetrievalService = mock(LeaderRetrievalService.class);
+			webRuntimeMonitor = new WebRuntimeMonitor(
+					config, leaderRetrievalService, actorSystem);
+
+			webRuntimeMonitor.start("akka://schmakka");
+
+			try (HttpTestClient client = new HttpTestClient(
+					"localhost", webRuntimeMonitor.getServerPort())) {
+
+				client.sendGetRequest("index.html", deadline.timeLeft());
+
+				HttpTestClient.SimpleHttpResponse response = client.getNextResponse();
+
+				assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE, response.getStatus());
+				assertEquals(MimeTypes.getMimeTypeForExtension("txt"), response.getType());
+				assertTrue(response.getContent().contains("refresh"));
+			}
+		}
+		finally {
+			if (actorSystem != null) {
+				actorSystem.shutdown();
+			}
+
+			if (webRuntimeMonitor != null) {
+				webRuntimeMonitor.stop();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private void waitForLeaderNotification(
+			ActorSystem system,
+			ActorRef expectedLeader,
+			JobManagerRetriever retriever,
+			Deadline deadline) throws Exception {
+
+		String expectedJobManagerUrl = AkkaUtils.getAkkaURL(system, expectedLeader);
+
+		while (deadline.hasTimeLeft()) {
+			ActorRef leaderRef = retriever.awaitJobManagerGatewayAndWebPort()._1().actor();
+
+			if (AkkaUtils.getAkkaURL(system, leaderRef).equals(expectedJobManagerUrl)) {
+				return;
+			}
+			else {
+				Thread.sleep(100);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
new file mode 100644
index 0000000..d7d4457
--- /dev/null
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.webmonitor.testutils;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.http.DefaultFullHttpRequest;
+import io.netty.handler.codec.http.HttpClientCodec;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpContentDecompressor;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpObject;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.util.CharsetUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A simple HTTP client.
+ *
+ * <pre>
+ * HttpTestClient client = new HttpTestClient("localhost", 8081);
+ * client.sendGetRequest("/overview", timeout);
+ * SimpleHttpResponse response = client.getNextResponse(timeout);
+ *
+ * assertEquals(200, response.getStatus().code()); // OK
+ * assertEquals("application/json", response.getType());
+ * assertTrue(response.getContent().contains("\"jobs-running\":0"));
+ * </pre>
+ *
+ * This code is based on Netty's HttpSnoopClient.
+ *
+ * @see <a href="https://github.com/netty/netty/blob/master/example/src/main/java/io/netty/example/http/snoop/HttpSnoopClient.java">HttpSnoopClient</a>
+ */
+public class HttpTestClient implements AutoCloseable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(HttpTestClient.class);
+
+	/** Target host */
+	private final String host;
+
+	/** Target port */
+	private final int port;
+
+	/** Netty's thread group for the client */
+	private final EventLoopGroup group;
+
+	/** Client bootstrap */
+	private final Bootstrap bootstrap;
+
+	/** Responses received by the client */
+	private final BlockingQueue<SimpleHttpResponse> responses = new LinkedBlockingQueue<>();
+
+	/**
+	 * Creates a client instance for the server at the target host and port.
+	 *
+	 * @param host Host of the HTTP server
+	 * @param port Port of the HTTP server
+	 */
+	public HttpTestClient(String host, int port) {
+		this.host = host;
+		this.port = port;
+
+		this.group = new NioEventLoopGroup();
+
+		this.bootstrap = new Bootstrap();
+		this.bootstrap.group(group)
+				.channel(NioSocketChannel.class)
+				.handler(new ChannelInitializer<SocketChannel>() {
+
+					@Override
+					protected void initChannel(SocketChannel ch) throws Exception {
+						ChannelPipeline p = ch.pipeline();
+						p.addLast(new HttpClientCodec());
+						p.addLast(new HttpContentDecompressor());
+						p.addLast(new ClientHandler(responses));
+					}
+				});
+	}
+
+	/**
+	 * Sends a request to to the server.
+	 *
+	 * <pre>
+	 * HttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/overview");
+	 * request.headers().set(HttpHeaders.Names.HOST, host);
+	 * request.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE);
+	 *
+	 * sendRequest(request);
+	 * </pre>
+	 *
+	 * @param request The {@link HttpRequest} to send to the server
+	 */
+	public void sendRequest(HttpRequest request, FiniteDuration timeout) throws InterruptedException, TimeoutException {
+		LOG.debug("Writing {}.", request);
+
+		// Make the connection attempt.
+		ChannelFuture connect = bootstrap.connect(host, port);
+
+		Channel channel;
+		if (connect.await(timeout.toMillis(), TimeUnit.MILLISECONDS)) {
+			channel = connect.channel();
+		}
+		else {
+			throw new TimeoutException("Connection failed");
+		}
+
+		channel.writeAndFlush(request);
+	}
+
+	/**
+	 * Sends a simple GET request to the given path. You only specify the $path part of
+	 * http://$host:$host/$path.
+	 *
+	 * @param path The $path to GET (http://$host:$host/$path)
+	 */
+	public void sendGetRequest(String path, FiniteDuration timeout) throws TimeoutException, InterruptedException {
+		if (!path.startsWith("/")) {
+			path = "/" + path;
+		}
+
+		HttpRequest getRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1,
+				HttpMethod.GET, path);
+		getRequest.headers().set(HttpHeaders.Names.HOST, host);
+		getRequest.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE);
+
+		sendRequest(getRequest, timeout);
+	}
+
+	/**
+	 * Returns the next available HTTP response. A call to this method blocks until a response
+	 * becomes available.
+	 *
+	 * @return The next available {@link SimpleHttpResponse}
+	 */
+	public SimpleHttpResponse getNextResponse() throws InterruptedException {
+		return responses.take();
+	}
+
+	/**
+	 * Returns the next available HTTP response . A call to this method blocks until a response
+	 * becomes available or throws an Exception if the timeout fires.
+	 *
+	 * @param timeout Timeout in milliseconds for the next response to become available
+	 * @return The next available {@link SimpleHttpResponse}
+	 */
+	public SimpleHttpResponse getNextResponse(FiniteDuration timeout) throws InterruptedException,
+			TimeoutException {
+
+		SimpleHttpResponse response = responses.poll(timeout.toMillis(), TimeUnit.MILLISECONDS);
+
+		if (response == null) {
+			throw new TimeoutException("No response within timeout of " + timeout + " ms");
+		}
+		else {
+			return response;
+		}
+	}
+
+	/**
+	 * Closes the client.
+	 */
+	@Override
+	public void close() throws InterruptedException {
+		if (group != null) {
+			group.shutdownGracefully();
+		}
+
+		LOG.debug("Closed");
+	}
+
+	/**
+	 * A simple HTTP response.
+	 */
+	public static class SimpleHttpResponse {
+
+		private final HttpResponseStatus status;
+
+		private final String type;
+
+		private final String content;
+
+		private final String location;
+
+		public SimpleHttpResponse(HttpResponseStatus status, String type, String content, String location) {
+			this.status = status;
+			this.type = type;
+			this.content = content;
+			this.location = location;
+		}
+
+		public HttpResponseStatus getStatus() {
+			return status;
+		}
+
+		public String getType() {
+			return type;
+		}
+
+		public final String getLocation() {
+			return location;
+		}
+
+		public String getContent() {
+			return content;
+		}
+
+		@Override
+		public String toString() {
+			return "HttpResponse(status=" + status + ", type='" + type + "'" + ", content='" +
+					content + "')";
+		}
+	}
+
+	/**
+	 * The response handler. Responses from the server are handled here.
+	 */
+	@ChannelHandler.Sharable
+	private static class ClientHandler extends SimpleChannelInboundHandler<HttpObject> {
+
+		private final BlockingQueue<SimpleHttpResponse> responses;
+
+		private HttpResponseStatus currentStatus;
+
+		private String currentType;
+
+		private String currentLocation;
+
+		private String currentContent = "";
+
+		public ClientHandler(BlockingQueue<SimpleHttpResponse> responses) {
+			this.responses = responses;
+		}
+
+		@Override
+		protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
+			LOG.debug("Received {}", msg);
+
+			if (msg instanceof HttpResponse) {
+				HttpResponse response = (HttpResponse) msg;
+
+				currentStatus = response.getStatus();
+				currentType = response.headers().get(HttpHeaders.Names.CONTENT_TYPE);
+				currentLocation = response.headers().get(HttpHeaders.Names.LOCATION);
+
+				if (HttpHeaders.isTransferEncodingChunked(response)) {
+					LOG.debug("Content is chunked");
+				}
+			}
+
+			if (msg instanceof HttpContent) {
+				HttpContent content = (HttpContent) msg;
+
+				// Add the content
+				currentContent += content.content().toString(CharsetUtil.UTF_8);
+
+				// Finished with this
+				if (content instanceof LastHttpContent) {
+					responses.add(new SimpleHttpResponse(currentStatus, currentType,
+							currentContent, currentLocation));
+
+					currentStatus = null;
+					currentType = null;
+					currentLocation = null;
+					currentContent = "";
+
+					ctx.close();
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/gulpfile.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/gulpfile.js b/flink-runtime-web/web-dashboard/gulpfile.js
index 737519e..2abfcce 100644
--- a/flink-runtime-web/web-dashboard/gulpfile.js
+++ b/flink-runtime-web/web-dashboard/gulpfile.js
@@ -43,7 +43,7 @@ var path = require('path');
 var environment = 'development';
 var paths = {
   src: './app/',
-  dest: './web/',
+  dest: '../src/main/resources/web/',
   vendor: './bower_components/',
   vendorLocal: './vendor-local/',
   assets: './assets/',
@@ -168,7 +168,7 @@ gulp.task('watch', function () {
 });
 
 gulp.task('serve', serve({
-  root: 'web',
+  root: '../src/main/resources/web/',
   port: 3001
 }));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/css/index.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/css/index.css b/flink-runtime-web/web-dashboard/web/css/index.css
deleted file mode 100644
index 43f7d98..0000000
--- a/flink-runtime-web/web-dashboard/web/css/index.css
+++ /dev/null
@@ -1,596 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#sidebar {
-  overflow: hidden;
-  position: fixed;
-  left: -250px;
-  top: 0;
-  bottom: 0;
-  height: 100%;
-  width: 250px;
-  background: #151515;
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-  -webkit-box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
-  box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
-}
-#sidebar.sidebar-visible {
-  left: 0;
-}
-#sidebar .logo {
-  width: auto;
-  height: 22px;
-}
-#sidebar .logo img {
-  display: inline-block;
-}
-#sidebar .navbar-static-top {
-  overflow: hidden;
-  height: 51px;
-}
-#sidebar .navbar-static-top .navbar-header {
-  width: 100%;
-}
-#sidebar .navbar-brand.navbar-brand-text {
-  font-size: 14px;
-  font-weight: bold;
-  color: #fff;
-  padding-left: 0;
-}
-#sidebar .nav > li > a {
-  color: #aaa;
-  margin-bottom: 1px;
-}
-#sidebar .nav > li > a:hover,
-#sidebar .nav > li > a:focus {
-  background-color: rgba(40,40,40,0.5);
-}
-#sidebar .nav > li > a.active {
-  background-color: rgba(100,100,100,0.5);
-}
-#content {
-  background-color: #fff;
-  overflow: hidden;
-  margin-left: 0;
-  padding-top: 70px;
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-}
-#content .navbar-main,
-#content .navbar-main-additional {
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-}
-#content .navbar-main-additional {
-  margin-top: 51px;
-  border-bottom: none;
-  padding: 0 20px;
-}
-#content .navbar-main-additional .nav-tabs {
-  margin: 0 -20px;
-  padding: 0 20px;
-}
-#content .navbar-secondary-additional {
-  border: none;
-  padding: 0 20px;
-  margin-bottom: 0;
-}
-#content .navbar-secondary-additional .nav-tabs {
-  margin: 0 -20px;
-}
-#content.sidebar-visible {
-  margin-left: 250px;
-}
-#content.sidebar-visible .navbar-main,
-#content.sidebar-visible .navbar-main-additional {
-  left: 250px;
-}
-#content #fold-button {
-  display: inline-block;
-  margin-left: 20px;
-}
-#content #content-inner {
-  padding: 0px 20px 20px 20px;
-}
-#content #content-inner.has-navbar-main-additional {
-  padding-top: 42px;
-}
-.page-header {
-  margin: 0 0 20px 0;
-}
-.nav > li > a,
-.nav > li > a:hover,
-.nav > li > a:focus {
-  color: #aaa;
-  background-color: transparent;
-  border-bottom: 2px solid transparent;
-}
-.nav > li.active > a,
-.nav > li.active > a:hover,
-.nav > li.active > a:focus {
-  color: #000;
-  border-bottom: 2px solid #000;
-}
-.nav.nav-tabs {
-  margin-bottom: 20px;
-}
-.table .table {
-  background-color: transparent;
-}
-.table th {
-  font-weight: normal;
-  color: #999;
-}
-.table td.td-long {
-  width: 20%;
-  white-space: pre-wrap;
-  white-space: -moz-pre-wrap;
-  white-space: -pre-wrap;
-  white-space: -o-pre-wrap;
-  word-wrap: break-word;
-}
-.table.table-clickable tr {
-  cursor: pointer;
-}
-.table.table-inner {
-  background-color: transparent;
-}
-.table.table-properties {
-  table-layout: fixed;
-  white-space: nowrap;
-}
-.table.table-properties td {
-  width: 50%;
-  white-space: nowrap;
-  overflow: hidden;
-  -o-text-overflow: ellipsis;
-  text-overflow: ellipsis;
-}
-.table.table-body-hover > tbody {
-  border-top: none;
-  border-left: 2px solid transparent;
-}
-.table.table-body-hover > tbody.active {
-  border-left: 2px solid #000;
-}
-.table.table-body-hover > tbody:hover td:not(.tab-column),
-.table.table-body-hover > tbody.active td:not(.tab-column) {
-  background-color: #f0f0f0;
-}
-.table.table-body-hover > tbody:hover td.tab-column li.active,
-.table.table-body-hover > tbody.active td.tab-column li.active {
-  background-color: #f0f0f0;
-}
-.table.table-activable th.tab-column,
-.table.table-activable td.tab-column {
-  border-top: none;
-  width: 47px;
-}
-.table.table-activable td.tab-column {
-  border-right: 1px solid #ddd;
-}
-.table.table-activable td {
-  position: relative;
-}
-.table .small-label {
-  text-transform: uppercase;
-  font-size: 13px;
-  color: #999;
-}
-.panel.panel-dashboard .huge {
-  font-size: 28px;
-}
-.panel.panel-lg {
-  font-size: 16px;
-}
-.panel.panel-lg .badge {
-  font-size: 14px;
-}
-.navbar-secondary {
-  overflow: auto;
-}
-.navbar-main .navbar-title,
-.navbar-secondary .navbar-title,
-.navbar-main-additional .navbar-title,
-.panel.panel-multi .navbar-title,
-.navbar-secondary-additional .navbar-title,
-.navbar-main .panel-title,
-.navbar-secondary .panel-title,
-.navbar-main-additional .panel-title,
-.panel.panel-multi .panel-title,
-.navbar-secondary-additional .panel-title {
-  float: left;
-  font-size: 18px;
-  padding: 12px 20px 13px 10px;
-  color: #333;
-  display: inline-block;
-}
-.navbar-main .navbar-info,
-.navbar-secondary .navbar-info,
-.navbar-main-additional .navbar-info,
-.panel.panel-multi .navbar-info,
-.navbar-secondary-additional .navbar-info,
-.navbar-main .panel-info,
-.navbar-secondary .panel-info,
-.navbar-main-additional .panel-info,
-.panel.panel-multi .panel-info,
-.navbar-secondary-additional .panel-info {
-  float: left;
-  font-size: 14px;
-  padding: 15px 15px 15px 15px;
-  color: #999;
-  display: inline-block;
-  border-right: 1px solid #e7e7e7;
-  overflow: hidden;
-}
-.navbar-main .navbar-info .overflow,
-.navbar-secondary .navbar-info .overflow,
-.navbar-main-additional .navbar-info .overflow,
-.panel.panel-multi .navbar-info .overflow,
-.navbar-secondary-additional .navbar-info .overflow,
-.navbar-main .panel-info .overflow,
-.navbar-secondary .panel-info .overflow,
-.navbar-main-additional .panel-info .overflow,
-.panel.panel-multi .panel-info .overflow,
-.navbar-secondary-additional .panel-info .overflow {
-  position: absolute;
-  display: block;
-  -o-text-overflow: ellipsis;
-  text-overflow: ellipsis;
-  overflow: hidden;
-  height: 22px;
-  line-height: 22px;
-  vertical-align: middle;
-}
-.navbar-main .navbar-info.first,
-.navbar-secondary .navbar-info.first,
-.navbar-main-additional .navbar-info.first,
-.panel.panel-multi .navbar-info.first,
-.navbar-secondary-additional .navbar-info.first,
-.navbar-main .panel-info.first,
-.navbar-secondary .panel-info.first,
-.navbar-main-additional .panel-info.first,
-.panel.panel-multi .panel-info.first,
-.navbar-secondary-additional .panel-info.first {
-  border-left: 1px solid #e7e7e7;
-}
-.navbar-main .navbar-info.last,
-.navbar-secondary .navbar-info.last,
-.navbar-main-additional .navbar-info.last,
-.panel.panel-multi .navbar-info.last,
-.navbar-secondary-additional .navbar-info.last,
-.navbar-main .panel-info.last,
-.navbar-secondary .panel-info.last,
-.navbar-main-additional .panel-info.last,
-.panel.panel-multi .panel-info.last,
-.navbar-secondary-additional .panel-info.last {
-  border-right: none;
-}
-.panel.panel-multi .panel-heading {
-  padding: 0;
-}
-.panel.panel-multi .panel-heading .panel-info.thin {
-  padding: 8px 10px;
-}
-.panel.panel-multi .panel-body {
-  padding: 10px;
-  background-color: #fdfdfd;
-  color: #999;
-  font-size: 13px;
-}
-.panel.panel-multi .panel-body.clean {
-  color: inherit;
-  font-size: inherit;
-}
-.navbar-main-additional,
-.navbar-secondary-additional {
-  min-height: 40px;
-  background-color: #fdfdfd;
-}
-.navbar-main-additional .navbar-info,
-.navbar-secondary-additional .navbar-info {
-  font-size: 13px;
-  padding: 10px 15px 10px 15px;
-}
-.nav-top-affix.affix {
-  width: 100%;
-  top: 50px;
-  margin-left: -20px;
-  padding-left: 20px;
-  margin-right: -20px;
-  padding-right: 20px;
-  background-color: #fff;
-  z-index: 1;
-}
-.badge-default[href]:hover,
-.badge-default[href]:focus {
-  background-color: #808080;
-}
-.badge-primary {
-  background-color: #428bca;
-}
-.badge-primary[href]:hover,
-.badge-primary[href]:focus {
-  background-color: #3071a9;
-}
-.badge-success {
-  background-color: #5cb85c;
-}
-.badge-success[href]:hover,
-.badge-success[href]:focus {
-  background-color: #449d44;
-}
-.badge-info {
-  background-color: #5bc0de;
-}
-.badge-info[href]:hover,
-.badge-info[href]:focus {
-  background-color: #31b0d5;
-}
-.badge-warning {
-  background-color: #f0ad4e;
-}
-.badge-warning[href]:hover,
-.badge-warning[href]:focus {
-  background-color: #ec971f;
-}
-.badge-danger {
-  background-color: #d9534f;
-}
-.badge-danger[href]:hover,
-.badge-danger[href]:focus {
-  background-color: #c9302c;
-}
-.indicator {
-  display: inline-block;
-  margin-right: 15px;
-}
-.indicator.indicator-primary {
-  color: #428bca;
-}
-.indicator.indicator-success {
-  color: #5cb85c;
-}
-.indicator.indicator-info {
-  color: #5bc0de;
-}
-.indicator.indicator-warning {
-  color: #f0ad4e;
-}
-.indicator.indicator-danger {
-  color: #d9534f;
-}
-pre.exception {
-  border: none;
-  background-color: transparent;
-  padding: 0;
-  margin: 0;
-}
-.nav-tabs.tabs-vertical {
-  position: absolute;
-  left: 0;
-  top: 0;
-  border-bottom: none;
-  z-index: 100;
-}
-.nav-tabs.tabs-vertical li {
-  float: none;
-  margin-bottom: 0;
-  margin-right: -1px;
-}
-.nav-tabs.tabs-vertical li > a {
-  margin-right: 0;
-  -webkit-border-radius: 0;
-  border-radius: 0;
-  border-bottom: none;
-  border-left: 2px solid transparent;
-}
-.nav-tabs.tabs-vertical li > a:hover,
-.nav-tabs.tabs-vertical li > a:focus {
-  border-bottom: none;
-  border-left: 2px solid #000;
-}
-.nav-tabs.tabs-vertical li.active > a {
-  border-bottom: none;
-  border-left: 2px solid #000;
-}
-.navbar-main .navbar-title,
-.navbar-secondary .navbar-title,
-.navbar-main-additional .navbar-title,
-.navbar-secondary-additional .navbar-title {
-  padding: 12px 20px 13px 20px;
-}
-livechart {
-  width: 30%;
-  height: 30%;
-  text-align: center;
-}
-.canvas-wrapper {
-  border: 1px solid #ddd;
-  position: relative;
-  margin-bottom: 20px;
-}
-.canvas-wrapper .main-canvas {
-  height: 400px;
-  overflow: hidden;
-}
-.canvas-wrapper .main-canvas .zoom-buttons {
-  position: absolute;
-  top: 10px;
-  right: 10px;
-}
-.label-group .label {
-  display: inline-block;
-  width: 2em;
-  padding-left: 0.1em;
-  padding-right: 0.1em;
-  margin: 0;
-  border-right: 1px solid #fff;
-  -webkit-border-radius: 0;
-  border-radius: 0;
-}
-.label-group .label.label-black {
-  background-color: #000;
-}
-svg.graph {
-  overflow: hidden;
-}
-svg.graph g.type-TK > rect {
-  fill: #00ffd0;
-}
-svg.graph text {
-  font-weight: 300;
-  font-size: 14px;
-}
-svg.graph .node {
-  cursor: pointer;
-}
-svg.graph .node > rect {
-  stroke: #999;
-  stroke-width: 5px;
-  fill: #fff;
-  margin: 0;
-  padding: 0;
-}
-svg.graph .node[active] > rect {
-  fill: #eee;
-}
-svg.graph .node.node-mirror > rect {
-  stroke: #a8a8a8;
-}
-svg.graph .node.node-iteration > rect {
-  stroke: #cd3333;
-}
-svg.graph .node.node-source > rect {
-  stroke: #4ce199;
-}
-svg.graph .node.node-sink > rect {
-  stroke: #e6ec8b;
-}
-svg.graph .node.node-normal > rect {
-  stroke: #3fb6d8;
-}
-svg.graph .node h4 {
-  color: #000;
-}
-svg.graph .node h5 {
-  color: #999;
-}
-svg.graph .edgeLabel rect {
-  fill: #fff;
-}
-svg.graph .edgePath path {
-  stroke: #333;
-  stroke-width: 2px;
-  fill: #333;
-}
-svg.graph .label {
-  color: #777;
-  margin: 0;
-}
-svg.graph .edge-label {
-  font-size: 14px;
-}
-svg.graph .node-label {
-  display: block;
-  margin: 0;
-  text-decoration: none;
-}
-.timeline {
-  overflow: hidden;
-}
-.timeline-canvas {
-  overflow: hidden;
-  padding: 10px;
-}
-.timeline-canvas .bar-container {
-  overflow: hidden;
-}
-.timeline-canvas .timeline-insidelabel,
-.timeline-canvas .timeline-series {
-  cursor: pointer;
-}
-.timeline-canvas.secondary .timeline-insidelabel,
-.timeline-canvas.secondary .timeline-series {
-  cursor: auto;
-}
-.qtip-timeline-bar {
-  font-size: 14px;
-  line-height: 1.4;
-}
-@media (min-width: 1024px) and (max-width: 1279px) {
-  #sidebar {
-    left: 0;
-    width: 160px;
-  }
-  #sidebar .navbar-static-top .navbar-brand-text {
-    display: none;
-  }
-  #content {
-    margin-left: 160px;
-  }
-  #content #fold-button {
-    display: none;
-  }
-  #content .navbar-main,
-  #content .navbar-main-additional {
-    left: 160px;
-  }
-  .table td.td-long {
-    width: 20%;
-  }
-}
-@media (min-width: 1280px) {
-  #sidebar {
-    left: 0;
-  }
-  #content {
-    margin-left: 250px;
-  }
-  #content #fold-button {
-    display: none;
-  }
-  #content .navbar-main,
-  #content .navbar-main-additional {
-    left: 250px;
-  }
-  .table td.td-long {
-    width: 30%;
-  }
-}
-#total-mem {
-  background-color: #7cb5ec;
-}
-#heap-mem {
-  background-color: #434348;
-}
-#non-heap-mem {
-  background-color: #90ed7d;
-}
-a.show-pointer {
-  cursor: pointer;
-}


[14/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf
deleted file mode 100644
index ed9372f..0000000
Binary files a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff
deleted file mode 100644
index 8b280b9..0000000
Binary files a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2 b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2
deleted file mode 100644
index 3311d58..0000000
Binary files a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/images/flink-logo.png
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/images/flink-logo.png b/flink-runtime-web/web-dashboard/web/images/flink-logo.png
deleted file mode 100644
index 088fb27..0000000
Binary files a/flink-runtime-web/web-dashboard/web/images/flink-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/index.html b/flink-runtime-web/web-dashboard/web/index.html
deleted file mode 100644
index a6df249..0000000
--- a/flink-runtime-web/web-dashboard/web/index.html
+++ /dev/null
@@ -1,55 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
---><!DOCTYPE html>
-<html lang="en">
-  <head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <title>Apache Flink Web Dashboard</title>
-    <link rel="stylesheet" href="css/vendor.css" type="text/css">
-    <link rel="stylesheet" href="css/index.css" type="text/css">
-    <script src="js/vendor.js"></script>
-    <script src="js/index.js"></script>
-  </head>
-  <body ng-app="flinkApp" ng-strict-di>
-    <div id="sidebar" ng-class="{ 'sidebar-visible': sidebarVisible }">
-      <nav class="navbar navbar-inverse navbar-static-top">
-        <div class="navbar-header"><a ui-sref="overview" class="navbar-brand"><img alt="Apache Flink Dashboard" src="images/flink-logo.png" class="logo"></a><a ui-sref="overview" class="navbar-brand navbar-brand-text">Apache Flink Dashboard</a></div>
-      </nav>
-      <div class="navbar navbar-sidebar">
-        <ul class="nav">
-          <li><a ui-sref="overview" ui-sref-active="active"><i class="fa fa-dashboard fa-fw"></i> 
-              Overview</a></li>
-          <li><a ui-sref="running-jobs" ui-sref-active="active"><i class="fa fa-tasks fa-fw"></i> 
-              Running Jobs</a></li>
-          <li><a ui-sref="completed-jobs" ui-sref-active="active"><i class="fa fa-check-circle fa-fw"></i> 
-              Completed Jobs</a></li>
-          <li><a ui-sref="all-manager" ui-sref-active="active"><i class="fa fa-sitemap fa-fw"></i> 
-              Task Managers</a></li>
-          <li><a ui-sref="jobmanager.config" ui-sref-active="active"><i class="fa fa-server fa-fw"></i> 
-              Job Manager</a></li>
-        </ul>
-      </div>
-    </div>
-    <div id="content" ng-class="{ 'sidebar-visible': sidebarVisible }">
-      <div ui-view="main"></div>
-    </div>
-  </body>
-</html>
\ No newline at end of file


[13/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/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
deleted file mode 100644
index d370765..0000000
--- a/flink-runtime-web/web-dashboard/web/js/index.js
+++ /dev/null
@@ -1,1420 +0,0 @@
-angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
-  $rootScope.sidebarVisible = false;
-  return $rootScope.showSidebar = function() {
-    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
-    return $rootScope.sidebarClass = 'force-show';
-  };
-}]).value('flinkConfig', {
-  "refresh-interval": 10000
-}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
-  MainService.loadConfig().then(function(config) {
-    angular.extend(flinkConfig, config);
-    JobsService.listJobs();
-    return $interval(function() {
-      return JobsService.listJobs();
-    }, flinkConfig["refresh-interval"]);
-  });
-  Highcharts.setOptions({
-    global: {
-      useUTC: false
-    }
-  });
-  Highcharts.createElement('link', {
-    href: '//fonts.googleapis.com/css?family=Dosis:400,600',
-    rel: 'stylesheet',
-    type: 'text/css'
-  }, null, document.getElementsByTagName('head')[0]);
-  Highcharts.theme = {
-    colors: ["#7cb5ec", "#f7a35c", "#90ee7e", "#7798BF", "#aaeeee", "#ff0066", "#eeaaee", "#55BF3B", "#DF5353", "#7798BF", "#aaeeee"],
-    chart: {
-      backgroundColor: null,
-      style: {
-        fontFamily: "Dosis, sans-serif"
-      }
-    },
-    title: {
-      style: {
-        fontSize: '16px',
-        fontWeight: 'bold',
-        textTransform: 'uppercase'
-      }
-    },
-    tooltip: {
-      borderWidth: 0,
-      backgroundColor: 'rgba(219,219,216,0.8)',
-      shadow: false
-    },
-    legend: {
-      itemStyle: {
-        fontWeight: 'bold',
-        fontSize: '13px'
-      }
-    },
-    xAxis: {
-      gridLineWidth: 1,
-      labels: {
-        style: {
-          fontSize: '12px'
-        }
-      }
-    },
-    yAxis: {
-      minorTickInterval: 'auto',
-      title: {
-        style: {
-          textTransform: 'uppercase'
-        }
-      },
-      labels: {
-        style: {
-          fontSize: '12px'
-        }
-      }
-    },
-    plotOptions: {
-      candlestick: {
-        lineColor: '#404048'
-      }
-    },
-    background2: '#F0F0EA'
-  };
-  return Highcharts.setOptions(Highcharts.theme);
-}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
-  return $uiViewScrollProvider.useAnchorScroll();
-}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
-  $stateProvider.state("overview", {
-    url: "/overview",
-    views: {
-      main: {
-        templateUrl: "partials/overview.html",
-        controller: 'OverviewController'
-      }
-    }
-  }).state("running-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: true,
-    views: {
-      main: {
-        templateUrl: "partials/jobs/job.html",
-        controller: 'SingleJobController'
-      }
-    }
-  }).state("single-job.plan", {
-    url: "",
-    abstract: true,
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.plan.html",
-        controller: 'JobPlanController'
-      }
-    }
-  }).state("single-job.plan.overview", {
-    url: "",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
-        controller: 'JobPlanOverviewController'
-      }
-    }
-  }).state("single-job.plan.accumulators", {
-    url: "/accumulators",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
-        controller: 'JobPlanAccumulatorsController'
-      }
-    }
-  }).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.statistics", {
-    url: "/statistics",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.statistics.html"
-      }
-    }
-  }).state("single-job.exceptions", {
-    url: "/exceptions",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.exceptions.html",
-        controller: 'JobExceptionsController'
-      }
-    }
-  }).state("single-job.properties", {
-    url: "/properties",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.properties.html",
-        controller: 'JobPropertiesController'
-      }
-    }
-  }).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}",
-    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.log", {
-    url: "/logfile",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
-      }
-    }
-  }).state("single-manager.stdout", {
-    url: "/stdout",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
-      }
-    }
-  }).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'
-      }
-    }
-  });
-  return $urlRouterProvider.otherwise("/overview");
-}]);
-
-angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
-  return {
-    transclude: true,
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: "@"
-    },
-    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'label label-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
-  return {
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: '@'
-    },
-    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('tableProperty', function() {
-  return {
-    replace: true,
-    scope: {
-      value: '='
-    },
-    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
-  };
-});
-
-angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
-  var amDurationFormatExtendedFilter;
-  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
-    if (typeof value === "undefined" || value === null) {
-      return "";
-    }
-    return moment.duration(value, format).format(durationFormat, {
-      trim: false
-    });
-  };
-  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
-  return amDurationFormatExtendedFilter;
-}]).filter("humanizeText", function() {
-  return function(text) {
-    if (text) {
-      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
-    } else {
-      return '';
-    }
-  };
-}).filter("bytes", function() {
-  return function(bytes, precision) {
-    var number, units;
-    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
-      return "-";
-    }
-    if (typeof precision === "undefined") {
-      precision = 1;
-    }
-    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
-    number = Math.floor(Math.log(bytes) / Math.log(1024));
-    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
-  };
-});
-
-angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("config").success(function(data, status, headers, config) {
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
-  return JobManagerConfigService.loadConfig().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['config'] = data;
-  });
-}]).controller('JobManagerLogsController', ["$scope", "JobManagerLogsService", function($scope, JobManagerLogsService) {
-  JobManagerLogsService.loadLogs().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['log'] = data;
-  });
-  return $scope.reloadData = function() {
-    return JobManagerLogsService.loadLogs().then(function(data) {
-      return $scope.jobmanager['log'] = data;
-    });
-  };
-}]).controller('JobManagerStdoutController', ["$scope", "JobManagerStdoutService", function($scope, JobManagerStdoutService) {
-  JobManagerStdoutService.loadStdout().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['stdout'] = data;
-  });
-  return $scope.reloadData = function() {
-    return JobManagerStdoutService.loadStdout().then(function(data) {
-      return $scope.jobmanager['stdout'] = data;
-    });
-  };
-}]);
-
-angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var config;
-  config = {};
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/config").success(function(data, status, headers, config) {
-      config = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('JobManagerLogsService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var logs;
-  logs = {};
-  this.loadLogs = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/log").success(function(data, status, headers, config) {
-      logs = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('JobManagerStdoutService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var stdout;
-  stdout = {};
-  this.loadStdout = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/stdout").success(function(data, status, headers, config) {
-      stdout = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('running');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
-  var refresher;
-  console.log('SingleJobController');
-  $scope.jobid = $stateParams.jobid;
-  $scope.job = null;
-  $scope.plan = null;
-  $scope.vertices = null;
-  JobsService.loadJob($stateParams.jobid).then(function(data) {
-    $scope.job = data;
-    $scope.plan = data.plan;
-    return $scope.vertices = data.vertices;
-  });
-  refresher = $interval(function() {
-    return JobsService.loadJob($stateParams.jobid).then(function(data) {
-      $scope.job = data;
-      return $scope.$broadcast('reload');
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    $scope.job = null;
-    $scope.plan = null;
-    $scope.vertices = null;
-    return $interval.cancel(refresher);
-  });
-}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobPlanController');
-  $scope.nodeid = null;
-  $scope.nodeUnfolded = false;
-  $scope.stateList = JobsService.stateList();
-  $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      $scope.accumulators = null;
-      return $scope.$broadcast('reload');
-    } else {
-      $scope.nodeid = null;
-      $scope.nodeUnfolded = false;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      return $scope.accumulators = null;
-    }
-  };
-  $scope.deactivateNode = function() {
-    $scope.nodeid = null;
-    $scope.nodeUnfolded = false;
-    $scope.vertex = null;
-    $scope.subtasks = null;
-    return $scope.accumulators = null;
-  };
-  return $scope.toggleFold = function() {
-    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
-  };
-}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanOverviewController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
-    JobsService.getSubtasks($scope.nodeid).then(function(data) {
-      return $scope.subtasks = data;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanOverviewController');
-    if ($scope.nodeid) {
-      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
-        return $scope.subtasks = data;
-      });
-    }
-  });
-}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanAccumulatorsController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
-    JobsService.getAccumulators($scope.nodeid).then(function(data) {
-      $scope.accumulators = data.main;
-      return $scope.subtaskAccumulators = data.subtasks;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanAccumulatorsController');
-    if ($scope.nodeid) {
-      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
-        $scope.accumulators = data.main;
-        return $scope.subtaskAccumulators = data.subtasks;
-      });
-    }
-  });
-}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobTimelineVertexController');
-  JobsService.getVertex($stateParams.vertexId).then(function(data) {
-    return $scope.vertex = data;
-  });
-  return $scope.$on('reload', function(event) {
-    console.log('JobTimelineVertexController');
-    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
-      return $scope.vertex = data;
-    });
-  });
-}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  return JobsService.loadExceptions().then(function(data) {
-    return $scope.exceptions = data;
-  });
-}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPropertiesController');
-  return $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      return JobsService.getNode(nodeid).then(function(data) {
-        return $scope.node = data;
-      });
-    } else {
-      $scope.nodeid = null;
-      return $scope.node = null;
-    }
-  };
-}]);
-
-angular.module('flinkApp').directive('vertex', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
-    scope: {
-      data: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data.subtasks, function(subtask, i) {
-          var times;
-          times = [
-            {
-              label: "Scheduled",
-              color: "#666",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["SCHEDULED"],
-              ending_time: subtask.timestamps["DEPLOYING"],
-              type: 'regular'
-            }, {
-              label: "Deploying",
-              color: "#aaa",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["DEPLOYING"],
-              ending_time: subtask.timestamps["RUNNING"],
-              type: 'regular'
-            }
-          ];
-          if (subtask.timestamps["FINISHED"] > 0) {
-            times.push({
-              label: "Running",
-              color: "#ddd",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["RUNNING"],
-              ending_time: subtask.timestamps["FINISHED"],
-              type: 'regular'
-            });
-          }
-          return testData.push({
-            label: "(" + subtask.subtask + ") " + subtask.host,
-            times: times
-          });
-        });
-        chart = d3.timeline().stack().tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("single").labelFormat(function(label) {
-          return label;
-        }).margin({
-          left: 100,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).relativeTime();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      analyzeTime(scope.data);
-    }
-  };
-}]).directive('timeline', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline' width='0' height='0'></svg>",
-    scope: {
-      vertices: "=",
-      jobid: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl, translateLabel;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      translateLabel = function(label) {
-        return label.replace("&gt;", ">");
-      };
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data, function(vertex) {
-          if (vertex['start-time'] > -1) {
-            if (vertex.type === 'scheduled') {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#cccccc",
-                    borderColor: "#555555",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    type: vertex.type
-                  }
-                ]
-              });
-            } else {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#d9f1f7",
-                    borderColor: "#62cdea",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    link: vertex.id,
-                    type: vertex.type
-                  }
-                ]
-              });
-            }
-          }
-        });
-        chart = d3.timeline().stack().click(function(d, i, datum) {
-          if (d.link) {
-            return $state.go("single-job.timeline.vertex", {
-              jobid: scope.jobid,
-              vertexId: d.link
-            });
-          }
-        }).tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("main").margin({
-          left: 0,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).showBorderLine().showHourTimeline();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      scope.$watch(attrs.vertices, function(data) {
-        if (data) {
-          return analyzeTime(data);
-        }
-      });
-    }
-  };
-}]).directive('jobPlan', ["$timeout", function($timeout) {
-  return {
-    template: "<svg class='graph' width='500' height='400'><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: '=',
-      setNode: '&'
-    },
-    link: function(scope, elem, attrs) {
-      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
-      g = null;
-      mainZoom = d3.behavior.zoom();
-      subgraphs = [];
-      jobid = attrs.jobid;
-      mainSvgElement = elem.children()[0];
-      mainG = elem.children().children()[0];
-      mainTmpElement = elem.children()[1];
-      d3mainSvg = d3.select(mainSvgElement);
-      d3mainSvgG = d3.select(mainG);
-      d3tmpSvg = d3.select(mainTmpElement);
-      containerW = elem.width();
-      angular.element(elem.children()[0]).width(containerW);
-      scope.zoomIn = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() < 2.99) {
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          mainZoom.scale(mainZoom.scale() + 0.1);
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      scope.zoomOut = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() > 0.31) {
-          mainZoom.scale(mainZoom.scale() - 0.1);
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      createLabelEdge = function(el) {
-        var labelValue;
-        labelValue = "";
-        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
-          labelValue += "<div class='edge-label'>";
-          if (el.ship_strategy != null) {
-            labelValue += el.ship_strategy;
-          }
-          if (el.temp_mode !== undefined) {
-            labelValue += " (" + el.temp_mode + ")";
-          }
-          if (el.local_strategy !== undefined) {
-            labelValue += ",<br>" + el.local_strategy;
-          }
-          labelValue += "</div>";
-        }
-        return labelValue;
-      };
-      isSpecialIterationNode = function(info) {
-        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
-      };
-      getNodeType = function(el, info) {
-        if (info === "mirror") {
-          return 'node-mirror';
-        } else if (isSpecialIterationNode(info)) {
-          return 'node-iteration';
-        } else {
-          return 'node-normal';
-        }
-      };
-      createLabelNode = function(el, info, maxW, maxH) {
-        var labelValue, stepName;
-        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
-        if (info === "mirror") {
-          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
-        } else {
-          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
-        }
-        if (el.description === "") {
-          labelValue += "";
-        } else {
-          stepName = el.description;
-          stepName = shortenString(stepName);
-          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
-        }
-        if (el.step_function != null) {
-          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
-        } else {
-          if (isSpecialIterationNode(info)) {
-            labelValue += "<h5>" + info + " Node</h5>";
-          }
-          if (el.parallelism !== "") {
-            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
-          }
-          if (el.operator !== undefined) {
-            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
-          }
-        }
-        labelValue += "</div>";
-        return labelValue;
-      };
-      extendLabelNodeForIteration = function(id, maxW, maxH) {
-        var labelValue, svgID;
-        svgID = "svg-" + id;
-        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
-        return labelValue;
-      };
-      shortenString = function(s) {
-        var sbr;
-        if (s.charAt(0) === "<") {
-          s = s.replace("<", "&lt;");
-          s = s.replace(">", "&gt;");
-        }
-        sbr = "";
-        while (s.length > 30) {
-          sbr = sbr + s.substring(0, 30) + "<br>";
-          s = s.substring(30, s.length);
-        }
-        sbr = sbr + s;
-        return sbr;
-      };
-      createNode = function(g, data, el, isParent, maxW, maxH) {
-        if (isParent == null) {
-          isParent = false;
-        }
-        if (el.id === data.partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "partialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "partialSolution")
-          });
-        } else if (el.id === data.next_partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextPartialSolution")
-          });
-        } else if (el.id === data.workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "workset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "workset")
-          });
-        } else if (el.id === data.next_workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextWorkset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextWorkset")
-          });
-        } else if (el.id === data.solution_set) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionSet", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionSet")
-          });
-        } else if (el.id === data.solution_delta) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionDelta", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionDelta")
-          });
-        } else {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "")
-          });
-        }
-      };
-      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
-        var missingNode;
-        if (existingNodes.indexOf(pred.id) !== -1) {
-          return g.setEdge(pred.id, el.id, {
-            label: createLabelEdge(pred),
-            labelType: 'html',
-            arrowhead: 'normal'
-          });
-        } else {
-          missingNode = searchForNode(data, pred.id);
-          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
-            missingNodes.push(missingNode.id);
-            g.setNode(missingNode.id, {
-              label: createLabelNode(missingNode, "mirror"),
-              labelType: 'html',
-              "class": getNodeType(missingNode, 'mirror')
-            });
-            return g.setEdge(missingNode.id, el.id, {
-              label: createLabelEdge(missingNode),
-              labelType: 'html'
-            });
-          }
-        }
-      };
-      loadJsonToDagre = function(g, data) {
-        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
-        existingNodes = [];
-        missingNodes = [];
-        if (data.nodes != null) {
-          toIterate = data.nodes;
-        } else {
-          toIterate = data.step_function;
-          isParent = true;
-        }
-        for (k = 0, len = toIterate.length; k < len; k++) {
-          el = toIterate[k];
-          maxW = 0;
-          maxH = 0;
-          if (el.step_function) {
-            sg = new dagreD3.graphlib.Graph({
-              multigraph: true,
-              compound: true
-            }).setGraph({
-              nodesep: 20,
-              edgesep: 0,
-              ranksep: 20,
-              rankdir: "LR",
-              marginx: 10,
-              marginy: 10
-            });
-            subgraphs[el.id] = sg;
-            loadJsonToDagre(sg, el);
-            r = new dagreD3.render();
-            d3tmpSvg.select('g').call(r, sg);
-            maxW = sg.graph().width;
-            maxH = sg.graph().height;
-            angular.element(mainTmpElement).empty();
-          }
-          createNode(g, data, el, isParent, maxW, maxH);
-          existingNodes.push(el.id);
-          if (el.inputs != null) {
-            ref = el.inputs;
-            for (l = 0, len1 = ref.length; l < len1; l++) {
-              pred = ref[l];
-              createEdge(g, data, el, existingNodes, pred, missingNodes);
-            }
-          }
-        }
-        return g;
-      };
-      searchForNode = function(data, nodeID) {
-        var el, i, j;
-        for (i in data.nodes) {
-          el = data.nodes[i];
-          if (el.id === nodeID) {
-            return el;
-          }
-          if (el.step_function != null) {
-            for (j in el.step_function) {
-              if (el.step_function[j].id === nodeID) {
-                return el.step_function[j];
-              }
-            }
-          }
-        }
-      };
-      drawGraph = function(data) {
-        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
-        g = new dagreD3.graphlib.Graph({
-          multigraph: true,
-          compound: true
-        }).setGraph({
-          nodesep: 70,
-          edgesep: 0,
-          ranksep: 50,
-          rankdir: "LR",
-          marginx: 40,
-          marginy: 40
-        });
-        loadJsonToDagre(g, data);
-        renderer = new dagreD3.render();
-        d3mainSvgG.call(renderer, g);
-        for (i in subgraphs) {
-          sg = subgraphs[i];
-          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
-        }
-        newScale = 0.5;
-        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
-        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
-        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
-        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
-        mainZoom.on("zoom", function() {
-          var ev;
-          ev = d3.event;
-          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
-        });
-        mainZoom(d3mainSvg);
-        return d3mainSvgG.selectAll('.node').on('click', function(d) {
-          return scope.setNode({
-            nodeid: d
-          });
-        });
-      };
-      scope.$watch(attrs.plan, function(newPlan) {
-        if (newPlan) {
-          return drawGraph(newPlan);
-        }
-      });
-    }
-  };
-}]);
-
-angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
-  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
-  currentJob = null;
-  currentPlan = null;
-  deferreds = {};
-  jobs = {
-    running: [],
-    finished: [],
-    cancelled: [],
-    failed: []
-  };
-  jobObservers = [];
-  notifyObservers = function() {
-    return angular.forEach(jobObservers, function(callback) {
-      return callback();
-    });
-  };
-  this.registerObserver = function(callback) {
-    return jobObservers.push(callback);
-  };
-  this.unRegisterObserver = function(callback) {
-    var index;
-    index = jobObservers.indexOf(callback);
-    return jobObservers.splice(index, 1);
-  };
-  this.stateList = function() {
-    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
-  };
-  this.translateLabelState = function(state) {
-    switch (state.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(list) {
-    return angular.forEach(list, function(item, jobKey) {
-      if (!(item['end-time'] > -1)) {
-        return item['end-time'] = item['start-time'] + item['duration'];
-      }
-    });
-  };
-  this.processVertices = function(data) {
-    angular.forEach(data.vertices, function(vertex, i) {
-      return vertex.type = 'regular';
-    });
-    return data.vertices.unshift({
-      name: 'Scheduled',
-      'start-time': data.timestamps['CREATED'],
-      'end-time': data.timestamps['CREATED'] + 1,
-      type: 'scheduled'
-    });
-  };
-  this.listJobs = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("joboverview").success((function(_this) {
-      return function(data, status, headers, config) {
-        angular.forEach(data, function(list, listKey) {
-          switch (listKey) {
-            case 'running':
-              return jobs.running = _this.setEndTimes(list);
-            case 'finished':
-              return jobs.finished = _this.setEndTimes(list);
-            case 'cancelled':
-              return jobs.cancelled = _this.setEndTimes(list);
-            case 'failed':
-              return jobs.failed = _this.setEndTimes(list);
-          }
-        });
-        deferred.resolve(jobs);
-        return notifyObservers();
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getJobs = function(type) {
-    return jobs[type];
-  };
-  this.getAllJobs = function() {
-    return jobs;
-  };
-  this.loadJob = function(jobid) {
-    currentJob = null;
-    deferreds.job = $q.defer();
-    $http.get("jobs/" + jobid).success((function(_this) {
-      return function(data, status, headers, config) {
-        _this.setEndTimes(data.vertices);
-        _this.processVertices(data);
-        return $http.get("jobs/" + jobid + "/config").success(function(jobConfig) {
-          data = angular.extend(data, jobConfig);
-          currentJob = data;
-          return deferreds.job.resolve(currentJob);
-        });
-      };
-    })(this));
-    return deferreds.job.promise;
-  };
-  this.getNode = function(nodeid) {
-    var deferred, seekNode;
-    seekNode = function(nodeid, data) {
-      var j, len, node, sub;
-      for (j = 0, len = data.length; j < len; j++) {
-        node = data[j];
-        if (node.id === nodeid) {
-          return node;
-        }
-        if (node.step_function) {
-          sub = seekNode(nodeid, node.step_function);
-        }
-        if (sub) {
-          return sub;
-        }
-      }
-      return null;
-    };
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var foundNode;
-        foundNode = seekNode(nodeid, currentJob.plan.nodes);
-        foundNode.vertex = _this.seekVertex(nodeid);
-        return deferred.resolve(foundNode);
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.seekVertex = function(nodeid) {
-    var j, len, ref, vertex;
-    ref = currentJob.vertices;
-    for (j = 0, len = ref.length; j < len; j++) {
-      vertex = ref[j];
-      if (vertex.id === nodeid) {
-        return vertex;
-      }
-    }
-    return null;
-  };
-  this.getVertex = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var vertex;
-        vertex = _this.seekVertex(vertexid);
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
-          vertex.subtasks = data.subtasks;
-          return deferred.resolve(vertex);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getSubtasks = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
-          var subtasks;
-          subtasks = data.subtasks;
-          return deferred.resolve(subtasks);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getAccumulators = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
-          var accumulators;
-          accumulators = data['user-accumulators'];
-          return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
-            var subtaskAccumulators;
-            subtaskAccumulators = data.subtasks;
-            return deferred.resolve({
-              main: accumulators,
-              subtasks: subtaskAccumulators
-            });
-          });
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.loadExceptions = function() {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
-          currentJob.exceptions = exceptions;
-          return deferred.resolve(exceptions);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
-  var refresh;
-  $scope.jobObserver = function() {
-    $scope.runningJobs = JobsService.getJobs('running');
-    return $scope.finishedJobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  $scope.jobObserver();
-  OverviewService.loadOverview().then(function(data) {
-    return $scope.overview = data;
-  });
-  refresh = $interval(function() {
-    return OverviewService.loadOverview().then(function(data) {
-      return $scope.overview = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]);
-
-angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var overview;
-  overview = {};
-  this.loadOverview = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("overview").success(function(data, status, headers, config) {
-      overview = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('AllTaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
-  var refresh;
-  TaskManagersService.loadManagers().then(function(data) {
-    return $scope.managers = data;
-  });
-  refresh = $interval(function() {
-    return TaskManagersService.loadManagers().then(function(data) {
-      return $scope.managers = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]).controller('SingleTaskManagerController', ["$scope", "$stateParams", "SingleTaskManagerService", "$interval", "flinkConfig", function($scope, $stateParams, SingleTaskManagerService, $interval, flinkConfig) {
-  var refresh;
-  $scope.metrics = {};
-  SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
-    return $scope.metrics = data[0];
-  });
-  refresh = $interval(function() {
-    return SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
-      return $scope.metrics = data[0];
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]);
-
-angular.module('flinkApp').directive('livechart', function() {
-  return {
-    link: function(scope, element, attrs) {
-      var getChartOptions, getChartType, getKey1, getKey2, getKey3, getKey4, getYAxisTitle, updateCharts;
-      getChartType = function() {
-        if (attrs.key === "cpuLoad") {
-          return "spline";
-        } else {
-          return "area";
-        }
-      };
-      getYAxisTitle = function() {
-        if (attrs.key === "cpuLoad") {
-          return "CPU Usage(%)";
-        } else {
-          return "Memory(MB)";
-        }
-      };
-      getKey1 = function() {
-        return "memory.total." + attrs.key;
-      };
-      getKey2 = function() {
-        return "memory.heap." + attrs.key;
-      };
-      getKey3 = function() {
-        return "memory.non-heap." + attrs.key;
-      };
-      getKey4 = function() {
-        return "cpuLoad";
-      };
-      getChartOptions = function() {
-        return {
-          title: {
-            text: ' '
-          },
-          chart: {
-            type: getChartType(),
-            zoomType: 'x'
-          },
-          xAxis: {
-            type: 'datetime'
-          },
-          yAxis: {
-            title: {
-              text: getYAxisTitle()
-            },
-            min: attrs.key === "cpuLoad" ? 0 : void 0,
-            max: attrs.key === "cpuLoad" ? 100 : void 0
-          },
-          series: [
-            {
-              name: "Memory: Total",
-              id: getKey1(),
-              data: [],
-              color: "#7cb5ec"
-            }, {
-              name: "Memory: Heap",
-              id: getKey2(),
-              data: [],
-              color: "#434348"
-            }, {
-              name: "Memory: Non-Heap",
-              id: getKey3(),
-              data: [],
-              color: "#90ed7d"
-            }, {
-              name: "CPU Usage",
-              id: getKey4(),
-              data: [],
-              color: "#f7a35c",
-              showInLegend: false
-            }
-          ],
-          legend: {
-            enabled: false
-          },
-          tooltip: {
-            shared: true
-          },
-          exporting: {
-            enabled: false
-          },
-          credits: {
-            enabled: false
-          }
-        };
-      };
-      if (element.highcharts() == null) {
-        element.highcharts(getChartOptions());
-      }
-      scope.$watch(attrs.data, function(value) {
-        return updateCharts(value);
-      });
-      return updateCharts = function(value) {
-        return (function(value) {
-          var chart, divider, heartbeat;
-          heartbeat = value.timeSinceLastHeartbeat;
-          chart = element.highcharts();
-          if (attrs.key === "cpuLoad") {
-            return chart.get(getKey4()).addPoint([heartbeat, +((value.metrics.gauges[getKey4()].value * 100).toFixed(2))], true, false);
-          } else {
-            divider = 1048576;
-            chart.get(getKey1()).addPoint([heartbeat, +((value.metrics.gauges[getKey1()].value / divider).toFixed(2))], true, false);
-            chart.get(getKey2()).addPoint([heartbeat, +((value.metrics.gauges[getKey2()].value / divider).toFixed(2))], true, false);
-            return chart.get(getKey3()).addPoint([heartbeat, +((value.metrics.gauges[getKey3()].value / divider).toFixed(2))], true, false);
-          }
-        })(value);
-      };
-    }
-  };
-});
-
-angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadManagers = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("taskmanagers").success(function(data, status, headers, config) {
-      return deferred.resolve(data['taskmanagers']);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('SingleTaskManagerService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadMetrics = function(taskmanagerid) {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("taskmanagers/" + taskmanagerid).success(function(data, status, headers, config) {
-      return deferred.resolve(data['taskmanagers']);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5
 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQUNILFlBQVksYUFBYSxLQUFLLFNBQUMsUUFBRDtJQUM1QixRQUFRLE9BQU8sYUFBYTtJQUU1QixZQUFZO0lDNUJaLE9EOEJBLFVBQVUsV0FBQTtNQzdCUixPRDhCQSxZQUFZO09BQ1osWUFBWTs7RUFFaEIsV0FBVyxXQUFXO0lBQ3BCLFFBQVE7TUFDTixRQUFROzs7RUFZWixXQUFXLGNBQWMsUUFBUTtJQUNoQyxNQUFNO0lBQ04sS0FBSztJQUNMLE1BQU07S0FDSixNQUFNLFNBQVMscUJBQXFCLFFBQVE7RUFFL0MsV0FBVyxRQUFRO0lBQ2xCLFFBQVEsQ0FBQyxXQUFXLFdBQVcsV0FB
 VyxXQUFXLFdBQVcsV0FBVyxXQUMxRSxXQUFXLFdBQVcsV0FBVztJQUNsQyxPQUFPO01BQ04saUJBQWlCO01BQ2pCLE9BQU87UUFDTixZQUFZOzs7SUFHZCxPQUFPO01BQ04sT0FBTztRQUNOLFVBQVU7UUFDVixZQUFZO1FBQ1osZUFBZTs7O0lBR2pCLFNBQVM7TUFDUixhQUFhO01BQ2IsaUJBQWlCO01BQ2pCLFFBQVE7O0lBRVQsUUFBUTtNQUNQLFdBQVc7UUFDVixZQUFZO1FBQ1osVUFBVTs7O0lBR1osT0FBTztNQUNOLGVBQWU7TUFDZixRQUFRO1FBQ1AsT0FBTztVQUNOLFVBQVU7Ozs7SUFJYixPQUFPO01BQ04sbUJBQW1CO01BQ25CLE9BQU87UUFDTixPQUFPO1VBQ04sZUFBZTs7O01BR2pCLFFBQVE7UUFDUCxPQUFPO1VBQ04sVUFBVTs7OztJQUliLGFBQWE7TUFDWixhQUFhO1FBQ1osV0FBVzs7O0lBSWIsYUFBYTs7RUN4Q2QsT0Q0Q0EsV0FBVyxXQUFXLFdBQVc7SUFLbEMsaUNBQU8sU0FBQyx1QkFBRDtFQy9DTixPRGdEQSxzQkFBc0I7SUFJdkIsZ0RBQU8sU0FBQyxnQkFBZ0Isb0JBQWpCO0VBQ04sZUFBZSxNQUFNLFlBQ25CO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGNBQ0w7SUFBQSxLQUFLO0lBQ
 0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLG1CQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSw0QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0NBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHVCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sOEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFFBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0g7SUFBQSxLQUFLO0
 lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRW5CLE1BQU0sMEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sY0FDSDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7OztLQUVwQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7Ozs7RUMxQmxCLE9ENEJBLG1CQUFtQixVQUFVOztBQzFCL0I7QUNoUEEsUUFBUSxPQUFPLFlBSWQsVUFBVSwyQkFBVyxTQUFDLGFBQUQ7RUNyQnBCLE9Ec0JBO0lBQUEsWUFBWTtJQUNaLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsaUJBQWlCLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJNUQsVUFBVSxvQ0F
 Bb0IsU0FBQyxhQUFEO0VDckI3QixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsc0NBQXNDLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJakYsVUFBVSxpQkFBaUIsV0FBQTtFQ3JCMUIsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLE9BQU87O0lBRVQsVUFBVTs7O0FDbEJaO0FDcEJBLFFBQVEsT0FBTyxZQUVkLE9BQU8sb0RBQTRCLFNBQUMscUJBQUQ7RUFDbEMsSUFBQTtFQUFBLGlDQUFpQyxTQUFDLE9BQU8sUUFBUSxnQkFBaEI7SUFDL0IsSUFBYyxPQUFPLFVBQVMsZUFBZSxVQUFTLE1BQXREO01BQUEsT0FBTzs7SUNoQlAsT0RrQkEsT0FBTyxTQUFTLE9BQU8sUUFBUSxPQUFPLGdCQUFnQjtNQUFFLE1BQU07OztFQUVoRSwrQkFBK0IsWUFBWSxvQkFBb0I7RUNmL0QsT0RpQkE7SUFFRCxPQUFPLGdCQUFnQixXQUFBO0VDakJ0QixPRGtCQSxTQUFDLE1BQUQ7SUFFRSxJQUFHLE1BQUg7TUNsQkUsT0RrQlcsS0FBSyxRQUFRLFNBQVMsS0FBSyxRQUFRLFdBQVU7V0FBMUQ7TUNoQkUsT0RnQmlFOzs7R0FFdEUsT0FBTyxTQUFTLFdBQUE7RUNkZixPRGVBLFNBQUMsT0FBTyxXQUFSO0lBQ0UsSUFBQSxRQUFBO0lBQUEsSUFBZSxNQUFNLFdBQVcsV0FBVyxDQUFJLFNBQVMsUUFBeEQ7TUFBQSxPQUFPOztJQUNQLElBQWtCLE9BQU8sY0FB
 YSxhQUF0QztNQUFBLFlBQVk7O0lBQ1osUUFBUSxDQUFFLFNBQVMsTUFBTSxNQUFNLE1BQU0sTUFBTTtJQUMzQyxTQUFTLEtBQUssTUFBTSxLQUFLLElBQUksU0FBUyxLQUFLLElBQUk7SUNUL0MsT0RVQSxDQUFDLFFBQVEsS0FBSyxJQUFJLE1BQU0sS0FBSyxNQUFNLFVBQVUsUUFBUSxhQUFhLE1BQU0sTUFBTTs7O0FDUGxGO0FDaEJBLFFBQVEsT0FBTyxZQUVkLFFBQVEsOENBQWUsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDdEIsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFVBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9Ec0JBOztBQ3BCRjtBQ09BLFFBQVEsT0FBTyxZQUVkLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQ25CeEMsT0RvQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7SUFDeEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNsQnRCLE9EbUJBLE9BQU8sV0FBVyxZQUFZOztJQUVqQyxXQUFXLGdFQUE0QixTQUFDLFFBQVEsdUJBQVQ7RUFDdEMsc0JBQXNCLFdBQVcsS0FBSyxTQUFDLE1BQUQ7SUFDcEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNqQnRCLE9Ea0JBLE9BQU8sV0FBVyxTQUFTOztFQ2hCN0IsT0RrQkEsT0FBTyxhQUFhLFdBQUE7SUNqQmxCLE9Ea0JBLHNCQUFzQixXQUFXLEtBQ
 UssU0FBQyxNQUFEO01DakJwQyxPRGtCQSxPQUFPLFdBQVcsU0FBUzs7O0lBRWhDLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQUN4Qyx3QkFBd0IsYUFBYSxLQUFLLFNBQUMsTUFBRDtJQUN4QyxJQUFJLE9BQUEsY0FBQSxNQUFKO01BQ0UsT0FBTyxhQUFhOztJQ2Z0QixPRGdCQSxPQUFPLFdBQVcsWUFBWTs7RUNkaEMsT0RnQkEsT0FBTyxhQUFhLFdBQUE7SUNmbEIsT0RnQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7TUNmeEMsT0RnQkEsT0FBTyxXQUFXLFlBQVk7Ozs7QUNacEM7QUNkQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDBEQUEyQixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNsQyxJQUFBO0VBQUEsU0FBUztFQUVULEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxxQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxTQUFTO01DcEJULE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9EcUJBO0lBRUQsUUFBUSx3REFBeUIsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDaEMsSUFBQTtFQUFBLE9BQU87RUFFUCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksa0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsT0FBTztNQ3RCUCxPRHVCQSxTQUFTLFFBQVE7O0lDckJuQixPRHVCQSxTQUFTOztFQ3JCWCxPRHVCQTtJQUVELFFBQVEsME
 RBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHFCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUN4QlQsT0R5QkEsU0FBUyxRQUFROztJQ3ZCbkIsT0R5QkEsU0FBUzs7RUN2QlgsT0R5QkE7O0FDdkJGO0FDdEJBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyxxSEFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQVksYUFBYSxXQUFyRTtFQUNqQyxJQUFBO0VBQUEsUUFBUSxJQUFJO0VBRVosT0FBTyxRQUFRLGFBQWE7RUFDNUIsT0FBTyxNQUFNO0VBQ2I
 sT0FBTyxPQUFPO0VBQ2QsT0FBTyxXQUFXO0VBRWxCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUFDM0MsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPLEtBQUs7SUMxQm5CLE9EMkJBLE9BQU8sV0FBVyxLQUFLOztFQUV6QixZQUFZLFVBQVUsV0FBQTtJQzFCcEIsT0QyQkEsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtNQUMzQyxPQUFPLE1BQU07TUMxQmIsT0Q0QkEsT0FBTyxXQUFXOztLQUVwQixZQUFZO0VDM0JkLE9ENkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUFDckIsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPO0lBQ2QsT0FBTyxXQUFXO0lDNUJsQixPRDhCQSxVQUFVLE9BQU87O0lBS3BCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDL0IsUUFBUSxJQUFJO0VBRVosT0FBTyxTQUFTO0VBQ2hCLE9BQU8sZUFBZTtFQUN0QixPQUFPLFlBQVksWUFBWTtFQUUvQixPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01BQ2hCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUFDbEIsT0FBTyxlQUFlO01DbEN0QixPRG9DQSxPQUFPLFdBQVc7V0FOcEI7TUFTRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxlQUFlO01BQ3RCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUNwQ2xCLE9EcUNBLE9BQU8sZUFBZTs7O0VBRTFCLE9BQU8saUJBQWlCLFdBQUE7SUFDdEIsT0FBTyxTQUFTO0lBQ2hCLE9BQU8s
 ZUFBZTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxXQUFXO0lDbkNsQixPRG9DQSxPQUFPLGVBQWU7O0VDbEN4QixPRG9DQSxPQUFPLGFBQWEsV0FBQTtJQ25DbEIsT0RvQ0EsT0FBTyxlQUFlLENBQUMsT0FBTzs7SUFJakMsV0FBVyx1REFBNkIsU0FBQyxRQUFRLGFBQVQ7RUFDdkMsUUFBUSxJQUFJO0VBRVosSUFBRyxPQUFPLFdBQVksQ0FBQyxPQUFPLFVBQVUsQ0FBQyxPQUFPLE9BQU8sS0FBdkQ7SUFDRSxZQUFZLFlBQVksT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01DdEMxQyxPRHVDQSxPQUFPLFdBQVc7OztFQ3BDdEIsT0RzQ0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQUNaLElBQUcsT0FBTyxRQUFWO01DckNFLE9Ec0NBLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUNyQzFDLE9Ec0NBLE9BQU8sV0FBVzs7OztJQUl6QixXQUFXLDJEQUFpQyxTQUFDLFFBQVEsYUFBVDtFQUMzQyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxlQUF2RDtJQUNFLFlBQVksZ0JBQWdCLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtNQUM5QyxPQUFPLGVBQWUsS0FBSztNQ3RDM0IsT0R1Q0EsT0FBTyxzQkFBc0IsS0FBSzs7O0VDcEN0QyxPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRL
 EtBQUssU0FBQyxNQUFEO1FBQzlDLE9BQU8sZUFBZSxLQUFLO1FDckMzQixPRHNDQSxPQUFPLHNCQUFzQixLQUFLOzs7O0lBSXpDLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDekMsUUFBUSxJQUFJO0VBRVosWUFBWSxVQUFVLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ3RDaEQsT0R1Q0EsT0FBTyxTQUFTOztFQ3JDbEIsT0R1Q0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQ3RDWixPRHVDQSxZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO01DdENoRCxPRHVDQSxPQUFPLFNBQVM7OztJQUlyQixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDdkNyQyxPRHdDQSxZQUFZLGlCQUFpQixLQUFLLFNBQUMsTUFBRDtJQ3ZDaEMsT0R3Q0EsT0FBTyxhQUFhOztJQUl2QixXQUFXLHFEQUEyQixTQUFDLFFBQVEsYUFBVDtFQUNyQyxRQUFRLElBQUk7RUN6Q1osT0QyQ0EsT0FBTyxhQUFhLFNBQUMsUUFBRDtJQUNsQixJQUFHLFdBQVUsT0FBTyxRQUFwQjtNQUNFLE9BQU8sU0FBUztNQzFDaEIsT0Q0Q0EsWUFBWSxRQUFRLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUMzQy9CLE9ENENBLE9BQU8sT0FBTzs7V0FKbEI7TUFPRSxPQUFPLFNBQVM7TUMzQ2hCLE9ENENBLE9BQU8sT0FBTzs7OztBQ3hDcEI7QUNuSEEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TU
 FBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBO01BQUEsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUztNQUVyQyxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsT0FBQSxLQUFBO1FBQUEsR0FBRyxPQUFPLE9BQU8sVUFBVSxLQUFLO1FBRWhDLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsU0FBUyxHQUFWO1VBQzdCLElBQUE7VUFBQSxRQUFRO1lBQ047Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNO2VBRVI7Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUFJVixJQUFHLFFBQVEsV0FBVyxjQUFjLEdBQXBDO1lBQ0UsTUFBTSxLQUFLO2NBQ1QsT0FBTztjQUNQLE9BQU87Y0FDUCxhQUFhO2NBQ2IsZUFBZSxRQUFRLFdBQVc7Y0FDbEMsYUFBYSxRQUFRLFdBQVc7Y0FDaEMsTUFBTTs7O1VDdEJSLE9EeUJGLFNBQVMsS0FBSztZQUNaLE9BQU8sTUFBSSxRQUFRLFVBQVEsT0FBSSxRQUFRO1lBQ3ZDLE9BQU87OztRQUdYLFFBQVEsR0FBRyxXQUFXLFFBQ3JCLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLFVBQVU7V0FFWCxPQUFPLFVBQ1AsWUFBWSxTQUFDLE9BQUQ7VUM1QlQ
 sT0Q2QkY7V0FFRCxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTtXQUM5QyxXQUFXLElBQ1g7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQ2hDckIsT0RpQ0E7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLFVBQVU7TUFDVixPQUFPOztJQUVULE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGlCQUFpQixTQUFDLE9BQUQ7UUNqQ2IsT0RrQ0YsTUFBTSxRQUFRLFFBQVE7O01BRXhCLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxNQUFNLFNBQUMsUUFBRDtVQUNwQixJQUFHLE9BQU8sZ0JBQWdCLENBQUMsR0FBM0I7WUFDRSxJQUFHLE9BQU8sU0FBUSxhQUFsQjtjQ2xDSSxPRG1DRixTQUFTLEtBQ1A7Z0JBQUEsT0FBTztrQkFDTDtvQkFBQSxPQUFPLGVBQWUsT0FBTztvQkFDN0IsT0FBTztvQkFDUCxhQUFhO29CQUNiLGVBQWUsT0FBTztvQkFDdEIsYUFBYSxPQUFPO29CQUNwQixNQUFNLE9BQU87Ozs7bUJBUm5CO2NDckJJLE9EZ0NGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29C
 QUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTztvQkFDYixNQUFNLE9BQU87Ozs7Ozs7UUFHdkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUFDbEMsSUFBRyxFQUFFLE1BQUw7WUMxQkksT0QyQkYsT0FBTyxHQUFHLDhCQUE4QjtjQUFFLE9BQU8sTUFBTTtjQUFPLFVBQVUsRUFBRTs7O1dBRzdFLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBR3ZCLFVBQVU7V0FFWCxPQUFPLFFBQ1AsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7V0FDNUMsV0FBVyxJQUNYLGlCQUNBO1FDMUJDLE9ENEJGLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUs7O01BRVIsTUFBTSxPQUFPLE1BQU0sVUFBVSxTQUFDLE1BQUQ7UUFDM0IsSUFBcUIsTUFBckI7VUM3QkksT0Q2QkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUM3QnBCLE9EOEJBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNO01BQ04sU0FBUzs7SUFFWCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsR0FBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLElBQUk7TUFDSixXQ
 UFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFLckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDMUN2QixPRDZDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM1Q3ZCLE9EK0NGLFdBQVcsS0FBSyxhQU
 FhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3RDZCxPRHVDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDeENyQixPRHlDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDeENJLE9EeUNGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUN6Q0QsT0QwQ0Y7ZUFERztVQ3ZDRCxPRDJDQTs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFFaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSx1QkFBdUIsUUFBUSxhQUFhLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHbEgsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLFdBQVc7ZUFEbkU7VUFHRSxjQUFjLDJCQUEyQixHQUFHLFdBQVc7O1FBQ3pELElBQUcsR0FBRyxnQkFBZSxJQUF
 yQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUF3RixHQUFHLGFBQVksV0FBdkc7WUFBQSxjQUFjLG9CQUFvQixjQUFjLEdBQUcscUJBQXFCOzs7UUFHMUUsY0FBYztRQzNDWixPRDRDRjs7TUFHRiw4QkFBOEIsU0FBQyxJQUFJLE1BQU0sTUFBWDtRQUM1QixJQUFBLFlBQUE7UUFBQSxRQUFRLFNBQVM7UUFFakIsYUFBYSxpQkFBaUIsUUFBUSxhQUFhLE9BQU8sYUFBYSxPQUFPO1FDNUM1RSxPRDZDRjs7TUFHRixnQkFBZ0IsU0FBQyxHQUFEO1FBRWQsSUFBQTtRQUFBLElBQUcsRUFBRSxPQUFPLE9BQU0sS0FBbEI7VUFDRSxJQUFJLEVBQUUsUUFBUSxLQUFLO1VBQ25CLElBQUksRUFBRSxRQUFRLEtBQUs7O1FBQ3JCLE1BQU07UUFDTixPQUFNLEVBQUUsU0FBUyxJQUFqQjtVQUNFLE1BQU0sTUFBTSxFQUFFLFVBQVUsR0FBRyxNQUFNO1VBQ2pDLElBQUksRUFBRSxVQUFVLElBQUksRUFBRTs7UUFDeEIsTUFBTSxNQUFNO1FDM0NWLE9ENENGOztNQUVGLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxVQUFrQixNQUFNLE1BQXRDO1FDM0NU
 LElBQUksWUFBWSxNQUFNO1VEMkNDLFdBQVc7O1FBRXBDLElBQUcsR0FBRyxPQUFNLEtBQUssa0JBQWpCO1VDekNJLE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLG1CQUFtQixNQUFNO1lBQ3BELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyx1QkFBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksdUJBQXVCLE1BQU07WUFDeEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLFNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLFdBQVcsTUFBTTtZQUM1QyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGdCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBT
 yxnQkFBZ0IsSUFBSSxpQkFBaUIsTUFBTTtZQUNsRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBSnRCO1VDbkNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLElBQUksTUFBTTtZQUNyQyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7Ozs7TUFFN0IsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTSxjQUFuQztRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3RDSSxPRHVDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFFdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxhQUFhLFFBQVEsWUFBWSxNQUFNLENBQUMsSUFBL0Q7WUFDRSxhQUFhLEtBQUssWUFBWTtZQUM5QixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ3RDaEMsT0R3Q0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsY0FBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBQ2hCLGVBQWU7UUFFZixJQUFHLE
 tBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUN2Q0ksS0FBSyxVQUFVO1VEd0NqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLFVBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDMUNJLE9BQU8sSUFBSTtjRDJDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTTs7OztRQ3RDakQsT0R3Q0Y7O01BR0YsZ0JBQWdCLFNBQUMsTUFBTSxRQUFQO1FBQ2QsSUFBQSxJQUFBLEdBQUE7UUFBQSxLQUFBLEtBQUEsS0FBQSxPQUFBO1VBQ0UsS0FBSyx
 LQUFLLE1BQU07VUFDaEIsSUFBYyxHQUFHLE9BQU0sUUFBdkI7WUFBQSxPQUFPOztVQUdQLElBQUcsR0FBQSxpQkFBQSxNQUFIO1lBQ0UsS0FBQSxLQUFBLEdBQUEsZUFBQTtjQUNFLElBQStCLEdBQUcsY0FBYyxHQUFHLE9BQU0sUUFBekQ7Z0JBQUEsT0FBTyxHQUFHLGNBQWM7Ozs7OztNQUVoQyxZQUFZLFNBQUMsTUFBRDtRQUNWLElBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQ2pDSSxLQUFLLFVBQVU7VURrQ2pCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdB
 QUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ25DTixPRG9DRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUFFckYsU0FBUztRQ25DUCxPRHFDRixXQUFXLFVBQVUsU0FBUyxHQUFHLFNBQVMsU0FBQyxHQUFEO1VDcEN0QyxPRHFDRixNQUFNLFFBQVE7WUFBRSxRQUFROzs7O01BRTVCLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDakNJLE9EaUNKLFVBQVU7Ozs7OztBQzNCaEI7QUMxYUEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBRWQsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3JCaEIsT0RzQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DckI1QixPRHNCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDcEJsQixPRHFCQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNuQjdCLE9Eb0JBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ25CWCxPRG9CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc
 0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDNUJILE9ENEJtQjtNQUR2QixLQUVPO1FDM0JILE9EMkJpQjtNQUZyQixLQUdPO1FDMUJILE9EMEJvQjtNQUh4QixLQUlPO1FDekJILE9EeUJvQjtNQUp4QixLQUtPO1FDeEJILE9Ed0JrQjtNQUx0QixLQU1PO1FDdkJILE9EdUJvQjtNQU54QixLQU9PO1FDdEJILE9Ec0JrQjtNQVB0QixLQVFPO1FDckJILE9EcUJnQjtNQVJwQjtRQ1hJLE9Eb0JHOzs7RUFFVCxLQUFDLGNBQWMsU0FBQyxNQUFEO0lDbEJiLE9EbUJBLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxRQUFQO01BQ3BCLElBQUEsRUFBTyxLQUFLLGNBQWMsQ0FBQyxJQUEzQjtRQ2xCRSxPRG1CQSxLQUFLLGNBQWMsS0FBSyxnQkFBZ0IsS0FBSzs7OztFQUVuRCxLQUFDLGtCQUFrQixTQUFDLE1BQUQ7SUFDakIsUUFBUSxRQUFRLEtBQUssVUFBVSxTQUFDLFFBQVEsR0FBVDtNQ2hCN0IsT0RpQkEsT0FBTyxPQUFPOztJQ2ZoQixPRGlCQSxLQUFLLFNBQVMsUUFBUTtNQUNwQixNQUFNO01BQ04sY0FBYyxLQUFLLFdBQVc7TUFDOUIsWUFBWSxLQUFLLFdBQVcsYUFBYTtNQUN6QyxNQUFNOzs7RUFHVixLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZUFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1
 lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxVQUFVLE9BQ25CLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFVBQVUsUUFBUSxXQUMzQixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE
 7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQyxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQzlELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RF
 RixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsVUFDbkQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDOUQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDOUQsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ES
 nlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUNwQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFlBQ1
 QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLDJGQUE2QixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDdkMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDakJkLE9EbUJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNsQnJCLE9EbUJBLFVBQVUsT0FBTzs7SUFFcEIsV0FBVyxrSEFBK0IsU0FBQyxRQUFRLGNBQWMsMEJBQTBCLFdBQVcsYUFBNUQ7RUFDekMsSUFBQTtFQUFBLE9BQU8sVUFBVTtFQUNqQix5QkFBeUIsWUFBWSxhQUFhLGVBQWUsS0FBSyxTQUFDLE1BQUQ7SUNqQnBFLE9Ea0JFLE9BQU8sVUFBVSxLQUFLOztFQUV4QixVQUFVLFVBQVUsV0FBQTtJQ2pCcEIsT0RrQkUseUJBQXlCLFlBQVksYUFBYSxlQUFlLEtBQUssU0FBQyxNQUFEO01DakJ0RSxPRGtCRSxPQUFPLFVBQVUsS0FBSzs7S0FDeEIsWUFBWTtFQ2hCaEIsT0RrQkUsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2pCdkIsT0RrQkUsVUFBVSxPQUFPOzs7QUNmdkI7QUNWQSxRQUFRLE9BQU8sWUFFZCxVQUFVLGFBQWEsV0FBQTtFQ25CdEI
 sT0RvQkE7SUFDRSxNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01BQ0osSUFBQSxpQkFBQSxjQUFBLFNBQUEsU0FBQSxTQUFBLFNBQUEsZUFBQTtNQUFBLGVBQWUsV0FBQTtRQUNiLElBQUcsTUFBTSxRQUFPLFdBQWhCO1VDbEJFLE9EbUJBO2VBREY7VUNoQkUsT0RtQkE7OztNQUVKLGdCQUFnQixXQUFBO1FBQ2QsSUFBRyxNQUFNLFFBQU8sV0FBaEI7VUNqQkUsT0RrQkE7ZUFERjtVQ2ZFLE9Ea0JBOzs7TUFFSixVQUFVLFdBQUE7UUNoQlIsT0RpQkEsa0JBQWtCLE1BQU07O01BQzFCLFVBQVUsV0FBQTtRQ2ZSLE9EZ0JBLGlCQUFpQixNQUFNOztNQUN6QixVQUFVLFdBQUE7UUNkUixPRGVBLHFCQUFxQixNQUFNOztNQUM3QixVQUFVLFdBQUE7UUNiUixPRGNBOztNQUVGLGtCQUFrQixXQUFBO1FDYmhCLE9EYXNCO1VBQ3RCLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFBQyxNQUFNO1lBQWdCLFVBQVU7O1VBQ3hDLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFDTCxPQUFPO2NBQUMsTUFBTTs7WUFDZCxLQUFVLE1BQU0sUUFBTyxZQUFsQixJQUFBLEtBQUE7WUFDTCxLQUFZLE1BQU0sUUFBTyxZQUFwQixNQUFBLEtBQUE7O1VBRVAsUUFBUTtZQUNOO2NBQUMsTUFBTTtjQUFpQixJQUFJO2NBQVcsTUFBTTtjQUFJLE9BQU87ZUFDeEQ7Y0FBQyxNQUFNO2NBQWdCLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztlQUN2RDtjQUFDLE1BQU07Y0FBb0IsSUFBSTtjQUFXLE1BQU07Y0FBSSxPQUFPO2VBQzNEO2NBQUMsTUFB
 TTtjQUFhLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztjQUFXLGNBQWM7OztVQUUvRSxRQUFRO1lBQUMsU0FBUzs7VUFDbEIsU0FBUztZQUFDLFFBQVE7O1VBQ2xCLFdBQVc7WUFBQyxTQUFTOztVQUNyQixTQUFTO1lBQUMsU0FBUzs7OztNQUdyQixJQUFJLFFBQUEsZ0JBQUEsTUFBSjtRQUNFLFFBQVEsV0FBVzs7TUFFckIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLE9BQUQ7UUN1QnZCLE9EdEJBLGFBQWE7O01Dd0JmLE9EckJBLGVBQWUsU0FBQyxPQUFEO1FDc0JiLE9EckJFLENBQUEsU0FBQyxPQUFEO1VBQ0EsSUFBQSxPQUFBLFNBQUE7VUFBQSxZQUFZLE1BQU07VUFDbEIsUUFBUSxRQUFRO1VBQ2hCLElBQUcsTUFBTSxRQUFPLFdBQWhCO1lDdUJFLE9EdEJBLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLEtBQUssUUFBUSxNQUNsRSxNQUFNO2lCQUhYO1lBS0UsVUFBVTtZQUNWLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLFNBQVMsUUFBUSxNQUN0RSxNQUFNO1lBQ1QsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQUFRLE1BQ3RFLE1BQU07WUNpQlQsT0RoQkEsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQ
 UFRLE1BQ3RFLE1BQU07O1dBakJWOzs7OztBQ3NDWDtBQzdGQSxRQUFRLE9BQU8sWUFFZCxRQUFRLHNEQUF1QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUM5QixLQUFDLGVBQWUsV0FBQTtJQUNkLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZ0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUSxLQUFLOztJQ25CeEIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7SUFFRCxRQUFRLDJEQUE0QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNuQyxLQUFDLGNBQWMsU0FBQyxlQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxrQkFBa0IsZUFDM0IsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DdEJQLE9EdUJBLFNBQVMsUUFBUSxLQUFLOztJQ3JCeEIsT0R1QkEsU0FBUzs7RUNyQlgsT0R1QkE7O0FDckJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbG
 VcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5ydW4gKCRyb290U2NvcGUpIC0+XHJcbiAgJHJvb3RTY29wZS5zaWR
 lYmFyVmlzaWJsZSA9IGZhbHNlXHJcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XHJcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcclxuICAgICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4udmFsdWUgJ2ZsaW5rQ29uZmlnJywge1xyXG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxyXG59XHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4ucnVuIChKb2JzU2VydmljZSwgTWFpblNlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIC0+XHJcbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cclxuICAgIGFuZ3VsYXIuZXh0ZW5kIGZsaW5rQ29uZmlnLCBjb25maWdcclxuXHJcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXHJcblxyXG4gICAgJGludGVydmFsIC0+XHJcbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcclxuICAgICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gIEhpZ2hjaGFydHMuc2V0T3B0aW9ucyh7XHJcbiAgICBnbG9iYWw6IHtcclxuICAgICAgdXNlVVRDOiBmYWxzZVxyXG4gICAgfVxyXG4gIH0pXHJcblxyXG4gICNcclxuICAjIEdyaWQtbGlnaHQgdGhlbWUgZm9yIEhpZ2hj
 aGFydHMgSlNcclxuICAjIEBhdXRob3IgVG9yc3RlaW4gSG9uc2lcclxuICAjXHJcbiAgIyBUYWtlbiBmcm9tIGh0dHBzOi8vZ2l0aHViLmNvbS9oaWdoc2xpZGUtc29mdHdhcmUvaGlnaGNoYXJ0cy5jb21cclxuICAjXHJcblxyXG5cclxuICBIaWdoY2hhcnRzLmNyZWF0ZUVsZW1lbnQoJ2xpbmsnLCB7XHJcbiAgXHRocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxyXG4gIFx0cmVsOiAnc3R5bGVzaGVldCcsXHJcbiAgXHR0eXBlOiAndGV4dC9jc3MnXHJcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XHJcblxyXG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XHJcbiAgXHRjb2xvcnM6IFtcIiM3Y2I1ZWNcIiwgXCIjZjdhMzVjXCIsIFwiIzkwZWU3ZVwiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCIsIFwiI2ZmMDA2NlwiLCBcIiNlZWFhZWVcIixcclxuICBcdFx0XCIjNTVCRjNCXCIsIFwiI0RGNTM1M1wiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCJdLFxyXG4gIFx0Y2hhcnQ6IHtcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiBudWxsLFxyXG4gIFx0XHRzdHlsZToge1xyXG4gIFx0XHRcdGZvbnRGYW1pbHk6IFwiRG9zaXMsIHNhbnMtc2VyaWZcIlxyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dGl0bGU6IHtcclxuICBcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRmb250U2l6ZTogJzE2cHgnLFxyXG4gIFx0XHRcd
 GZvbnRXZWlnaHQ6ICdib2xkJyxcclxuICBcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dG9vbHRpcDoge1xyXG4gIFx0XHRib3JkZXJXaWR0aDogMCxcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiAncmdiYSgyMTksMjE5LDIxNiwwLjgpJyxcclxuICBcdFx0c2hhZG93OiBmYWxzZVxyXG4gIFx0fSxcclxuICBcdGxlZ2VuZDoge1xyXG4gIFx0XHRpdGVtU3R5bGU6IHtcclxuICBcdFx0XHRmb250V2VpZ2h0OiAnYm9sZCcsXHJcbiAgXHRcdFx0Zm9udFNpemU6ICcxM3B4J1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0eEF4aXM6IHtcclxuICBcdFx0Z3JpZExpbmVXaWR0aDogMSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHR5QXhpczoge1xyXG4gIFx0XHRtaW5vclRpY2tJbnRlcnZhbDogJ2F1dG8nLFxyXG4gIFx0XHR0aXRsZToge1xyXG4gIFx0XHRcdHN0eWxlOiB7XHJcbiAgXHRcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHRcdH1cclxuICBcdFx0fSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHRwbG90T3B0aW9ucz
 oge1xyXG4gIFx0XHRjYW5kbGVzdGljazoge1xyXG4gIFx0XHRcdGxpbmVDb2xvcjogJyM0MDQwNDgnXHJcbiAgXHRcdH1cclxuICBcdH0sXHJcblxyXG4gIFx0YmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xyXG5cclxuICB9O1xyXG5cclxuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoSGlnaGNoYXJ0cy50aGVtZSk7XHJcblxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxyXG4gICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cclxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXHJcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxyXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgIG1haW46XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5
 nLWpvYnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcclxuICBcclxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxyXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgbWFpbjpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxyXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcclxuICAgIHVybDogXCJcIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIixcclxuICAgIHVybDogXCJc
 IlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0Lm92ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLFxyXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcclxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInIFxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXHJcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxyXG4gICAgdXJsOiBcIi97dmVydGV4SWR9XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICB2ZXJ0ZXg6XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4L
 mh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLFxyXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IuZXhjZXB0aW9uc1wiLFxyXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIixcclxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOl
 xyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiYWxsLW1hbmFnZXJcIixcclxuICAgIHVybDogXCIvdGFza21hbmFnZXJzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL2luZGV4Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL3Rhc2ttYW5hZ2VyL3t0YXNrbWFuYWdlcmlkfVwiXHJcbiAgICAgIHZpZXdzOlxyXG4gICAgICAgIG1haW46XHJcbiAgICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5odG

<TRUNCATED>

[04/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
index 1330b66..f6ee5c5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
@@ -18,14 +18,13 @@
 
 package org.apache.flink.runtime.checkpoint;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete;
 import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint;
@@ -34,13 +33,25 @@ import org.junit.Test;
 import java.util.Iterator;
 import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 /**
  * Tests for the checkpoint coordinator.
  */
 public class CheckpointCoordinatorTest {
-	
+
 	private static final ClassLoader cl = Thread.currentThread().getContextClassLoader();
-	
+
 	@Test
 	public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() {
 		try {
@@ -50,7 +61,7 @@ public class CheckpointCoordinatorTest {
 			// create some mock Execution vertices that receive the checkpoint trigger messages
 			ExecutionVertex triggerVertex1 = mock(ExecutionVertex.class);
 			ExecutionVertex triggerVertex2 = mock(ExecutionVertex.class);
-			
+
 			// create some mock Execution vertices that need to ack the checkpoint
 			final ExecutionAttemptID ackAttemptID1 = new ExecutionAttemptID();
 			final ExecutionAttemptID ackAttemptID2 = new ExecutionAttemptID();
@@ -59,10 +70,12 @@ public class CheckpointCoordinatorTest {
 
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 1, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 					new ExecutionVertex[] { ackVertex1, ackVertex2 },
-					new ExecutionVertex[] {}, cl );
+					new ExecutionVertex[] {}, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 
 			// nothing should be happening
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
@@ -103,10 +116,12 @@ public class CheckpointCoordinatorTest {
 
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 1, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 					new ExecutionVertex[] { ackVertex1, ackVertex2 },
-					new ExecutionVertex[] {}, cl );
+					new ExecutionVertex[] {}, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 
 			// nothing should be happening
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
@@ -138,17 +153,18 @@ public class CheckpointCoordinatorTest {
 			final ExecutionAttemptID triggerAttemptID2 = new ExecutionAttemptID();
 			ExecutionVertex triggerVertex1 = mockExecutionVertex(triggerAttemptID1);
 			ExecutionVertex triggerVertex2 = mockExecutionVertex(triggerAttemptID2);
-			
+
 			// create some mock Execution vertices that receive the checkpoint trigger messages
 			ExecutionVertex ackVertex1 = mock(ExecutionVertex.class);
 			ExecutionVertex ackVertex2 = mock(ExecutionVertex.class);
 
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 1, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 					new ExecutionVertex[] { ackVertex1, ackVertex2 },
-					new ExecutionVertex[] {}, cl );
+					new ExecutionVertex[] {}, cl, new StandaloneCheckpointIDCounter(), new
+					StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 
 			// nothing should be happening
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
@@ -168,13 +184,13 @@ public class CheckpointCoordinatorTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testTriggerAndConfirmSimpleCheckpoint() {
 		try {
 			final JobID jid = new JobID();
 			final long timestamp = System.currentTimeMillis();
-			
+
 			// create some mock Execution vertices that receive the checkpoint trigger messages
 			final ExecutionAttemptID attemptID1 = new ExecutionAttemptID();
 			final ExecutionAttemptID attemptID2 = new ExecutionAttemptID();
@@ -183,24 +199,26 @@ public class CheckpointCoordinatorTest {
 
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 1, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { vertex1, vertex2 },
 					new ExecutionVertex[] { vertex1, vertex2 },
-					new ExecutionVertex[] { vertex1, vertex2 }, cl);
-			
+					new ExecutionVertex[] { vertex1, vertex2 }, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
+
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
-			
+
 			// trigger the first checkpoint. this should succeed
 			assertTrue(coord.triggerCheckpoint(timestamp));
-			
+
 			// validate that we have a pending checkpoint
 			assertEquals(1, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
-			
+
 			long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey();
 			PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId);
-			
+
 			assertNotNull(checkpoint);
 			assertEquals(checkpointId, checkpoint.getCheckpointId());
 			assertEquals(timestamp, checkpoint.getCheckpointTimestamp());
@@ -210,7 +228,7 @@ public class CheckpointCoordinatorTest {
 			assertEquals(0, checkpoint.getCollectedStates().size());
 			assertFalse(checkpoint.isDiscarded());
 			assertFalse(checkpoint.isFullyAcknowledged());
-			
+
 			// check that the vertices received the trigger checkpoint message
 			{
 				TriggerCheckpoint expectedMessage1 = new TriggerCheckpoint(jid, attemptID1, checkpointId, timestamp);
@@ -218,7 +236,7 @@ public class CheckpointCoordinatorTest {
 				verify(vertex1, times(1)).sendMessageToCurrentExecution(eq(expectedMessage1), eq(attemptID1));
 				verify(vertex2, times(1)).sendMessageToCurrentExecution(eq(expectedMessage2), eq(attemptID2));
 			}
-			
+
 			// acknowledge from one of the tasks
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId));
 			assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks());
@@ -233,15 +251,15 @@ public class CheckpointCoordinatorTest {
 
 			// acknowledge the other task.
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId));
-			
+
 			// the checkpoint is internally converted to a successful checkpoint and the
 			// pending checkpoint object is disposed
 			assertTrue(checkpoint.isDiscarded());
-			
+
 			// the now we should have a completed checkpoint
 			assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints());
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
-			
+
 			// validate that the relevant tasks got a confirmation message
 			{
 				NotifyCheckpointComplete confirmMessage1 = new NotifyCheckpointComplete(jid, attemptID1, checkpointId, timestamp);
@@ -249,13 +267,13 @@ public class CheckpointCoordinatorTest {
 				verify(vertex1, times(1)).sendMessageToCurrentExecution(eq(confirmMessage1), eq(attemptID1));
 				verify(vertex2, times(1)).sendMessageToCurrentExecution(eq(confirmMessage2), eq(attemptID2));
 			}
-			
-			SuccessfulCheckpoint success = coord.getSuccessfulCheckpoints().get(0);
+
+			CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0);
 			assertEquals(jid, success.getJobId());
 			assertEquals(timestamp, success.getTimestamp());
 			assertEquals(checkpoint.getCheckpointId(), success.getCheckpointID());
 			assertTrue(success.getStates().isEmpty());
-			
+
 			// ---------------
 			// trigger another checkpoint and see that this one replaces the other checkpoint
 			// ---------------
@@ -265,11 +283,11 @@ public class CheckpointCoordinatorTest {
 			long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey();
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew));
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew));
-			
+
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints());
-			
-			SuccessfulCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0);
+
+			CompletedCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0);
 			assertEquals(jid, successNew.getJobId());
 			assertEquals(timestampNew, successNew.getTimestamp());
 			assertEquals(checkpointIdNew, successNew.getCheckpointID());
@@ -295,8 +313,7 @@ public class CheckpointCoordinatorTest {
 			fail(e.getMessage());
 		}
 	}
-	
-	
+
 	@Test
 	public void testMultipleConcurrentCheckpoints() {
 		try {
@@ -305,7 +322,7 @@ public class CheckpointCoordinatorTest {
 			final long timestamp2 = timestamp1 + 8617;
 
 			// create some mock execution vertices
-			
+
 			final ExecutionAttemptID triggerAttemptID1 = new ExecutionAttemptID();
 			final ExecutionAttemptID triggerAttemptID2 = new ExecutionAttemptID();
 
@@ -314,23 +331,25 @@ public class CheckpointCoordinatorTest {
 			final ExecutionAttemptID ackAttemptID3 = new ExecutionAttemptID();
 
 			final ExecutionAttemptID commitAttemptID = new ExecutionAttemptID();
-			
+
 			ExecutionVertex triggerVertex1 = mockExecutionVertex(triggerAttemptID1);
 			ExecutionVertex triggerVertex2 = mockExecutionVertex(triggerAttemptID2);
 
 			ExecutionVertex ackVertex1 = mockExecutionVertex(ackAttemptID1);
 			ExecutionVertex ackVertex2 = mockExecutionVertex(ackAttemptID2);
 			ExecutionVertex ackVertex3 = mockExecutionVertex(ackAttemptID3);
-			
+
 			ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID);
-			
+
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 2, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 					new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 },
-					new ExecutionVertex[] { commitVertex }, cl);
-			
+					new ExecutionVertex[] { commitVertex }, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE);
+
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
 
@@ -339,7 +358,7 @@ public class CheckpointCoordinatorTest {
 
 			assertEquals(1, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
-			
+
 			PendingCheckpoint pending1 = coord.getPendingCheckpoints().values().iterator().next();
 			long checkpointId1 = pending1.getCheckpointId();
 
@@ -348,10 +367,10 @@ public class CheckpointCoordinatorTest {
 					new TriggerCheckpoint(jid, triggerAttemptID1, checkpointId1, timestamp1), triggerAttemptID1);
 			verify(triggerVertex2, times(1)).sendMessageToCurrentExecution(
 					new TriggerCheckpoint(jid, triggerAttemptID2, checkpointId1, timestamp1), triggerAttemptID2);
-			
+
 			// acknowledge one of the three tasks
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1));
-			
+
 			// start the second checkpoint
 			// trigger the first checkpoint. this should succeed
 			assertTrue(coord.triggerCheckpoint(timestamp2));
@@ -373,23 +392,23 @@ public class CheckpointCoordinatorTest {
 					new TriggerCheckpoint(jid, triggerAttemptID1, checkpointId2, timestamp2), triggerAttemptID1);
 			verify(triggerVertex2, times(1)).sendMessageToCurrentExecution(
 					new TriggerCheckpoint(jid, triggerAttemptID2, checkpointId2, timestamp2), triggerAttemptID2);
-			
+
 			// we acknowledge the remaining two tasks from the first
 			// checkpoint and two tasks from the second checkpoint
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1));
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2));
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1));
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2));
-			
+
 			// now, the first checkpoint should be confirmed
 			assertEquals(1, coord.getNumberOfPendingCheckpoints());
 			assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints());
 			assertTrue(pending1.isDiscarded());
-			
+
 			// the first confirm message should be out
 			verify(commitVertex, times(1)).sendMessageToCurrentExecution(
 					new NotifyCheckpointComplete(jid, commitAttemptID, checkpointId1, timestamp1), commitAttemptID);
-			
+
 			// send the last remaining ack for the second checkpoint
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2));
 
@@ -401,17 +420,17 @@ public class CheckpointCoordinatorTest {
 			// the second commit message should be out
 			verify(commitVertex, times(1)).sendMessageToCurrentExecution(
 					new NotifyCheckpointComplete(jid, commitAttemptID, checkpointId2, timestamp2), commitAttemptID);
-			
+
 			// validate the committed checkpoints
-			List<SuccessfulCheckpoint> scs = coord.getSuccessfulCheckpoints();
-			
-			SuccessfulCheckpoint sc1 = scs.get(0);
+			List<CompletedCheckpoint> scs = coord.getSuccessfulCheckpoints();
+
+			CompletedCheckpoint sc1 = scs.get(0);
 			assertEquals(checkpointId1, sc1.getCheckpointID());
 			assertEquals(timestamp1, sc1.getTimestamp());
 			assertEquals(jid, sc1.getJobId());
 			assertTrue(sc1.getStates().isEmpty());
-			
-			SuccessfulCheckpoint sc2 = scs.get(1);
+
+			CompletedCheckpoint sc2 = scs.get(1);
 			assertEquals(checkpointId2, sc2.getCheckpointID());
 			assertEquals(timestamp2, sc2.getTimestamp());
 			assertEquals(jid, sc2.getJobId());
@@ -453,10 +472,12 @@ public class CheckpointCoordinatorTest {
 
 			// set up the coordinator and validate the initial state
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 10, 600000,
+					jid, 600000,
 					new ExecutionVertex[] { triggerVertex1, triggerVertex2 },
 					new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 },
-					new ExecutionVertex[] { commitVertex }, cl);
+					new ExecutionVertex[] { commitVertex }, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(10, cl), RecoveryMode.STANDALONE);
 
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
@@ -513,13 +534,13 @@ public class CheckpointCoordinatorTest {
 			// into a successful checkpoint
 			assertTrue(pending1.isDiscarded());
 			assertTrue(pending2.isDiscarded());
-			
+
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints());
 
 			// validate the committed checkpoints
-			List<SuccessfulCheckpoint> scs = coord.getSuccessfulCheckpoints();
-			SuccessfulCheckpoint success = scs.get(0);
+			List<CompletedCheckpoint> scs = coord.getSuccessfulCheckpoints();
+			CompletedCheckpoint success = scs.get(0);
 			assertEquals(checkpointId2, success.getCheckpointID());
 			assertEquals(timestamp2, success.getTimestamp());
 			assertEquals(jid, success.getJobId());
@@ -531,7 +552,7 @@ public class CheckpointCoordinatorTest {
 
 			// send the last remaining ack for the first checkpoint. This should not do anything
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1));
-			
+
 			coord.shutdown();
 		}
 		catch (Exception e) {
@@ -539,8 +560,7 @@ public class CheckpointCoordinatorTest {
 			fail(e.getMessage());
 		}
 	}
-	
-	
+
 	@Test
 	public void testCheckpointTimeoutIsolated() {
 		try {
@@ -565,22 +585,24 @@ public class CheckpointCoordinatorTest {
 
 			// set up the coordinator
 			// the timeout for the checkpoint is a 200 milliseconds
-			
+
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 2, 200,
+					jid, 200,
 					new ExecutionVertex[] { triggerVertex },
 					new ExecutionVertex[] { ackVertex1, ackVertex2 },
-					new ExecutionVertex[] { commitVertex }, cl);
-			
+					new ExecutionVertex[] { commitVertex }, cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE);
+
 			// trigger a checkpoint, partially acknowledged
 			assertTrue(coord.triggerCheckpoint(timestamp));
 			assertEquals(1, coord.getNumberOfPendingCheckpoints());
-			
+
 			PendingCheckpoint checkpoint = coord.getPendingCheckpoints().values().iterator().next();
 			assertFalse(checkpoint.isDiscarded());
-			
+
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId()));
-			
+
 			// wait until the checkpoint must have expired.
 			// we check every 250 msecs conservatively for 5 seconds
 			// to give even slow build servers a very good chance of completing this
@@ -591,7 +613,7 @@ public class CheckpointCoordinatorTest {
 			while (!checkpoint.isDiscarded() &&
 					coord.getNumberOfPendingCheckpoints() > 0 &&
 					System.currentTimeMillis() < deadline);
-			
+
 			assertTrue("Checkpoint was not canceled by the timeout", checkpoint.isDiscarded());
 			assertEquals(0, coord.getNumberOfPendingCheckpoints());
 			assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints());
@@ -599,7 +621,7 @@ public class CheckpointCoordinatorTest {
 			// no confirm message must have been sent
 			verify(commitVertex, times(0))
 					.sendMessageToCurrentExecution(any(NotifyCheckpointComplete.class), any(ExecutionAttemptID.class));
-			
+
 			coord.shutdown();
 		}
 		catch (Exception e) {
@@ -607,7 +629,7 @@ public class CheckpointCoordinatorTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void handleMessagesForNonExistingCheckpoints() {
 		try {
@@ -625,27 +647,28 @@ public class CheckpointCoordinatorTest {
 			ExecutionVertex ackVertex1 = mockExecutionVertex(ackAttemptID1);
 			ExecutionVertex ackVertex2 = mockExecutionVertex(ackAttemptID2);
 			ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID);
-			
+
 			CheckpointCoordinator coord = new CheckpointCoordinator(
-					jid, 2, 200000,
+					jid, 200000,
 					new ExecutionVertex[] { triggerVertex },
 					new ExecutionVertex[] { ackVertex1, ackVertex2 },
-					new ExecutionVertex[] { commitVertex }, cl);
+					new ExecutionVertex[] { commitVertex }, cl, new StandaloneCheckpointIDCounter
+					(), new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE);
 
 			assertTrue(coord.triggerCheckpoint(timestamp));
-			
+
 			long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next();
-			
+
 			// send some messages that do not belong to either the job or the any
 			// of the vertices that need to be acknowledged.
 			// non of the messages should throw an exception
-			
+
 			// wrong job id
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(new JobID(), ackAttemptID1, checkpointId));
-			
+
 			// unknown checkpoint
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, 1L));
-			
+
 			// unknown ack vertex
 			coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, new ExecutionAttemptID(), checkpointId));
 
@@ -660,15 +683,16 @@ public class CheckpointCoordinatorTest {
 	private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) {
 		return mockExecutionVertex(attemptID, ExecutionState.RUNNING);
 	}
-	
+
 	private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID, ExecutionState state) {
 		final Execution exec = mock(Execution.class);
 		when(exec.getAttemptId()).thenReturn(attemptID);
 		when(exec.getState()).thenReturn(state);
 
 		ExecutionVertex vertex = mock(ExecutionVertex.class);
+		when(vertex.getJobvertexId()).thenReturn(new JobVertexID());
 		when(vertex.getCurrentExecutionAttempt()).thenReturn(exec);
-		
+
 		return vertex;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java
new file mode 100644
index 0000000..96c4eea
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.junit.Assert.assertEquals;
+
+public abstract class CheckpointIDCounterTest extends TestLogger {
+
+	protected abstract CheckpointIDCounter createCompletedCheckpoints() throws Exception;
+
+	public static class StandaloneCheckpointIDCounterTest extends CheckpointIDCounterTest {
+
+		@Override
+		protected CheckpointIDCounter createCompletedCheckpoints() throws Exception {
+			return new StandaloneCheckpointIDCounter();
+		}
+	}
+
+	public static class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTest {
+
+		private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+		@AfterClass
+		public static void tearDown() throws Exception {
+			if (ZooKeeper != null) {
+				ZooKeeper.shutdown();
+			}
+		}
+
+		@Before
+		public void cleanUp() throws Exception {
+			ZooKeeper.deleteAll();
+		}
+
+		@Override
+		protected CheckpointIDCounter createCompletedCheckpoints() throws Exception {
+			return new ZooKeeperCheckpointIDCounter(ZooKeeper.getClient(),
+					"/checkpoint-id-counter");
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Tests serial increment and get calls.
+	 */
+	@Test
+	public void testSerialIncrementAndGet() throws Exception {
+		final CheckpointIDCounter counter = createCompletedCheckpoints();
+
+		try {
+			counter.start();
+
+			assertEquals(1, counter.getAndIncrement());
+			assertEquals(2, counter.getAndIncrement());
+			assertEquals(3, counter.getAndIncrement());
+			assertEquals(4, counter.getAndIncrement());
+		}
+		finally {
+			counter.stop();
+		}
+	}
+
+	/**
+	 * Tests concurrent increment and get calls from multiple Threads and verifies that the numbers
+	 * counts strictly increasing.
+	 */
+	@Test
+	public void testConcurrentGetAndIncrement() throws Exception {
+		// Config
+		final int numThreads = 8;
+
+		// Setup
+		final CountDownLatch startLatch = new CountDownLatch(1);
+		final CheckpointIDCounter counter = createCompletedCheckpoints();
+		counter.start();
+
+		ExecutorService executor = null;
+		try {
+			executor = Executors.newFixedThreadPool(numThreads);
+
+			List<Future<List<Long>>> resultFutures = new ArrayList<>(numThreads);
+
+			for (int i = 0; i < numThreads; i++) {
+				resultFutures.add(executor.submit(new Incrementer(startLatch, counter)));
+			}
+
+			// Kick off the incrementing
+			startLatch.countDown();
+
+			final int expectedTotal = numThreads * Incrementer.NumIncrements;
+
+			List<Long> all = new ArrayList<>(expectedTotal);
+
+			// Get the counts
+			for (Future<List<Long>> result : resultFutures) {
+				List<Long> counts = result.get();
+
+				for (long val : counts) {
+					all.add(val);
+				}
+			}
+
+			// Verify
+			Collections.sort(all);
+
+			assertEquals(expectedTotal, all.size());
+
+			long current = 0;
+			for (long val : all) {
+				// Incrementing counts
+				assertEquals(++current, val);
+			}
+
+			// The final count
+			assertEquals(expectedTotal + 1, counter.getAndIncrement());
+		}
+		finally {
+			if (executor != null) {
+				executor.shutdown();
+			}
+
+			counter.stop();
+		}
+	}
+
+	/**
+	 * Task repeatedly incrementing the {@link CheckpointIDCounter}.
+	 */
+	private static class Incrementer implements Callable<List<Long>> {
+
+		/** Total number of {@link CheckpointIDCounter#getAndIncrement()} calls. */
+		private final static int NumIncrements = 128;
+
+		private final CountDownLatch startLatch;
+
+		private final CheckpointIDCounter counter;
+
+		public Incrementer(CountDownLatch startLatch, CheckpointIDCounter counter) {
+			this.startLatch = startLatch;
+			this.counter = counter;
+		}
+
+		@Override
+		public List<Long> call() throws Exception {
+			final Random rand = new Random();
+			final List<Long> counts = new ArrayList<>();
+
+			// Wait for the main thread to kick off execution
+			this.startLatch.await();
+
+			for (int i = 0; i < NumIncrements; i++) {
+				counts.add(counter.getAndIncrement());
+
+				// To get some "random" interleaving ;)
+				Thread.sleep(rand.nextInt(20));
+			}
+
+			return counts;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
index 08cb0a3..32c15bf 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.LocalStateHandle;
 import org.apache.flink.runtime.state.StateHandle;
@@ -79,10 +80,12 @@ public class CheckpointStateRestoreTest {
 			map.put(statelessId, stateless);
 			
 			
-			CheckpointCoordinator coord = new CheckpointCoordinator(jid, 1, 200000L, 
+			CheckpointCoordinator coord = new CheckpointCoordinator(jid, 200000L,
 					new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
 					new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
-					new ExecutionVertex[0], cl);
+					new ExecutionVertex[0], cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 			
 			// create ourselves a checkpoint with state
 			final long timestamp = 34623786L;
@@ -148,10 +151,12 @@ public class CheckpointStateRestoreTest {
 			map.put(statelessId, stateless);
 
 
-			CheckpointCoordinator coord = new CheckpointCoordinator(jid, 1, 200000L,
+			CheckpointCoordinator coord = new CheckpointCoordinator(jid, 200000L,
 					new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
 					new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 },
-					new ExecutionVertex[0], cl);
+					new ExecutionVertex[0], cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 
 			// create ourselves a checkpoint with state
 			final long timestamp = 34623786L;
@@ -188,10 +193,12 @@ public class CheckpointStateRestoreTest {
 	@Test
 	public void testNoCheckpointAvailable() {
 		try {
-			CheckpointCoordinator coord = new CheckpointCoordinator(new JobID(), 1, 200000L,
+			CheckpointCoordinator coord = new CheckpointCoordinator(new JobID(), 200000L,
 					new ExecutionVertex[] { mock(ExecutionVertex.class) },
 					new ExecutionVertex[] { mock(ExecutionVertex.class) },
-					new ExecutionVertex[0], cl);
+					new ExecutionVertex[0], cl,
+					new StandaloneCheckpointIDCounter(),
+					new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE);
 
 			try {
 				coord.restoreLatestCheckpointedState(new HashMap<JobVertexID, ExecutionJobVertex>(), true, false);

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
new file mode 100644
index 0000000..9e3c605
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java
@@ -0,0 +1,297 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.CheckpointMessagesTest;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for basic {@link CompletedCheckpointStore} contract.
+ */
+public abstract class CompletedCheckpointStoreTest extends TestLogger {
+
+	/**
+	 * Creates the {@link CompletedCheckpointStore} implementation to be tested.
+	 */
+	protected abstract CompletedCheckpointStore createCompletedCheckpoints(
+			int maxNumberOfCheckpointsToRetain, ClassLoader userLoader) throws Exception;
+
+	// ---------------------------------------------------------------------------------------------
+
+	// Verify that discarded checkpoints are called with the correct class loader
+	private final ClassLoader userClassLoader = ClassLoader.getSystemClassLoader();
+
+	/**
+	 * Tests that at least one checkpoint needs to be retained.
+	 */
+	@Test(expected = Exception.class)
+	public void testExceptionOnNoRetainedCheckpoints() throws Exception {
+		createCompletedCheckpoints(0, userClassLoader);
+	}
+
+	/**
+	 * Tests adding and getting a checkpoint.
+	 */
+	@Test
+	public void testAddAndGetLatestCheckpoint() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader);
+
+		// Empty state
+		assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints());
+		assertEquals(0, checkpoints.getAllCheckpoints().size());
+
+		TestCheckpoint[] expected = new TestCheckpoint[] {
+				createCheckpoint(0), createCheckpoint(1) };
+
+		// Add and get latest
+		checkpoints.addCheckpoint(expected[0]);
+		assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints());
+		verifyCheckpoint(expected[0], checkpoints.getLatestCheckpoint());
+
+		checkpoints.addCheckpoint(expected[1]);
+		assertEquals(2, checkpoints.getNumberOfRetainedCheckpoints());
+		verifyCheckpoint(expected[1], checkpoints.getLatestCheckpoint());
+	}
+
+	/**
+	 * Tests that adding more checkpoints than retained discards the correct checkpoints (using
+	 * the correct class loader).
+	 */
+	@Test
+	public void testAddCheckpointMoreThanMaxRetained() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(1, userClassLoader);
+
+		TestCheckpoint[] expected = new TestCheckpoint[] {
+				createCheckpoint(0), createCheckpoint(1),
+				createCheckpoint(2), createCheckpoint(3)
+		};
+
+		// Add checkpoints
+		checkpoints.addCheckpoint(expected[0]);
+		assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints());
+
+		for (int i = 1; i < expected.length; i++) {
+			checkpoints.addCheckpoint(expected[i]);
+
+			// The ZooKeeper implementation discards asynchronously
+			expected[i - 1].awaitDiscard();
+			assertTrue(expected[i - 1].isDiscarded());
+			assertEquals(userClassLoader, expected[i - 1].getDiscardClassLoader());
+
+			assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints());
+		}
+	}
+
+	/**
+	 * Tests that
+	 * <ul>
+	 * <li>{@link CompletedCheckpointStore#getLatestCheckpoint()} returns <code>null</code>,</li>
+	 * <li>{@link CompletedCheckpointStore#getAllCheckpoints()} returns an empty list,</li>
+	 * <li>{@link CompletedCheckpointStore#getNumberOfRetainedCheckpoints()} returns 0.</li>
+	 * </ul>
+	 */
+	@Test
+	public void testEmptyState() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(1, userClassLoader);
+
+		assertNull(checkpoints.getLatestCheckpoint());
+		assertEquals(0, checkpoints.getAllCheckpoints().size());
+		assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints());
+	}
+
+	/**
+	 * Tests that all added checkpoints are returned.
+	 */
+	@Test
+	public void testGetAllCheckpoints() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader);
+
+		TestCheckpoint[] expected = new TestCheckpoint[] {
+				createCheckpoint(0), createCheckpoint(1),
+				createCheckpoint(2), createCheckpoint(3)
+		};
+
+		for (TestCheckpoint checkpoint : expected) {
+			checkpoints.addCheckpoint(checkpoint);
+		}
+
+		List<CompletedCheckpoint> actual = checkpoints.getAllCheckpoints();
+
+		assertEquals(expected.length, actual.size());
+
+		for (int i = 0; i < expected.length; i++) {
+			assertEquals(expected[i], actual.get(i));
+		}
+	}
+
+	/**
+	 * Tests that all checkpoints are discarded (using the correct class loader).
+	 */
+	@Test
+	public void testDiscardAllCheckpoints() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader);
+
+		TestCheckpoint[] expected = new TestCheckpoint[] {
+				createCheckpoint(0), createCheckpoint(1),
+				createCheckpoint(2), createCheckpoint(3)
+		};
+
+		for (TestCheckpoint checkpoint : expected) {
+			checkpoints.addCheckpoint(checkpoint);
+		}
+
+		checkpoints.discardAllCheckpoints();
+
+		// Empty state
+		assertNull(checkpoints.getLatestCheckpoint());
+		assertEquals(0, checkpoints.getAllCheckpoints().size());
+		assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints());
+
+		// All have been discarded
+		for (TestCheckpoint checkpoint : expected) {
+			// The ZooKeeper implementation discards asynchronously
+			checkpoint.awaitDiscard();
+			assertTrue(checkpoint.isDiscarded());
+			assertEquals(userClassLoader, checkpoint.getDiscardClassLoader());
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	protected TestCheckpoint createCheckpoint(int id) throws IOException {
+		return createCheckpoint(id, 4);
+	}
+
+	protected TestCheckpoint createCheckpoint(int id, int numberOfStates)
+			throws IOException {
+
+		JobVertexID jvid = new JobVertexID();
+
+		ArrayList<StateForTask> taskStates = new ArrayList<>();
+
+		for (int i = 0; i < numberOfStates; i++) {
+			SerializedValue<StateHandle<?>> stateHandle = new SerializedValue<StateHandle<?>>(
+					new CheckpointMessagesTest.MyHandle());
+
+			taskStates.add(new StateForTask(stateHandle, jvid, i));
+		}
+
+		return new TestCheckpoint(new JobID(), id, 0, taskStates);
+	}
+
+	private void verifyCheckpoint(CompletedCheckpoint expected, CompletedCheckpoint actual) {
+		assertEquals(expected.getJobId(), actual.getJobId());
+		assertEquals(expected.getCheckpointID(), actual.getCheckpointID());
+		assertEquals(expected.getTimestamp(), actual.getTimestamp());
+
+		List<StateForTask> expectedStates = expected.getStates();
+		List<StateForTask> actualStates = actual.getStates();
+
+		assertEquals(expectedStates.size(), actualStates.size());
+
+		for (int i = 0; i < expectedStates.size(); i++) {
+			assertEquals(expectedStates.get(i), actualStates.get(i));
+		}
+	}
+
+	/**
+	 * A test {@link CompletedCheckpoint}. We want to verify that the correct class loader is
+	 * used when discarding. Spying on a regular {@link CompletedCheckpoint} instance with
+	 * Mockito doesn't work, because it it breaks serializability.
+	 */
+	protected static class TestCheckpoint extends CompletedCheckpoint {
+
+		private static final long serialVersionUID = 4211419809665983026L;
+
+		private boolean isDiscarded;
+
+		// Latch for test variants which discard asynchronously
+		private transient final CountDownLatch discardLatch = new CountDownLatch(1);
+
+		private transient ClassLoader discardClassLoader;
+
+		public TestCheckpoint(
+				JobID jobId,
+				long checkpointId,
+				long timestamp,
+				ArrayList<StateForTask> states) {
+
+			super(jobId, checkpointId, timestamp, states);
+		}
+
+		@Override
+		public void discard(ClassLoader userClassLoader) {
+			super.discard(userClassLoader);
+
+			if (!isDiscarded) {
+				this.discardClassLoader = userClassLoader;
+				this.isDiscarded = true;
+
+				if (discardLatch != null) {
+					discardLatch.countDown();
+				}
+			}
+		}
+
+		public boolean isDiscarded() {
+			return isDiscarded;
+		}
+
+		public void awaitDiscard() throws InterruptedException {
+			if (discardLatch != null) {
+				discardLatch.await();
+			}
+		}
+
+		public ClassLoader getDiscardClassLoader() {
+			return discardClassLoader;
+		}
+
+		@Override
+		public boolean equals(Object o) {
+			if (this == o) return true;
+			if (o == null || getClass() != o.getClass()) return false;
+
+			TestCheckpoint that = (TestCheckpoint) o;
+
+			return getJobId().equals(that.getJobId())
+					&& getCheckpointID() == that.getCheckpointID();
+		}
+
+		@Override
+		public int hashCode() {
+			return getJobId().hashCode() + (int) getCheckpointID();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java
new file mode 100644
index 0000000..beccbf8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.checkpoint;
+
+/**
+ * Tests for basic {@link CompletedCheckpointStore} contract.
+ */
+public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointStoreTest {
+
+	@Override
+	protected CompletedCheckpointStore createCompletedCheckpoints(
+			int maxNumberOfCheckpointsToRetain,
+			ClassLoader userClassLoader) throws Exception {
+
+		return new StandaloneCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userClassLoader);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
new file mode 100644
index 0000000..4c6ddfd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
@@ -0,0 +1,101 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.runtime.state.LocalStateHandle;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for basic {@link CompletedCheckpointStore} contract and ZooKeeper state handling.
+ */
+public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpointStoreTest {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static String CheckpointsPath = "/checkpoints";
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (ZooKeeper != null) {
+			ZooKeeper.shutdown();
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		ZooKeeper.deleteAll();
+	}
+
+	@Override
+	protected CompletedCheckpointStore createCompletedCheckpoints(int maxNumberOfCheckpointsToRetain,
+			ClassLoader userLoader) throws Exception {
+
+		return new ZooKeeperCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userLoader,
+				ZooKeeper.createClient(), CheckpointsPath, new LocalStateHandle
+				.LocalStateHandleProvider<CompletedCheckpoint>());
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Tests that older checkpoints are cleaned up at startup.
+	 */
+	@Test
+	public void testRecover() throws Exception {
+		CompletedCheckpointStore checkpoints = createCompletedCheckpoints(3, ClassLoader
+				.getSystemClassLoader());
+
+		TestCheckpoint[] expected = new TestCheckpoint[] {
+				createCheckpoint(0), createCheckpoint(1), createCheckpoint(2)
+		};
+
+		// Add multiple checkpoints
+		checkpoints.addCheckpoint(expected[0]);
+		checkpoints.addCheckpoint(expected[1]);
+		checkpoints.addCheckpoint(expected[2]);
+
+		// All three should be in ZK
+		assertEquals(3, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size());
+
+		// Recover
+		checkpoints.recover();
+
+		// Only the latest one should be in ZK
+		Deadline deadline = new FiniteDuration(1, TimeUnit.MINUTES).fromNow();
+
+		// Retry this operation, because removal is asynchronous
+		while (deadline.hasTimeLeft() && ZooKeeper.getClient()
+				.getChildren().forPath(CheckpointsPath).size() != 1) {
+
+			Thread.sleep(Math.min(100, deadline.timeLeft().toMillis()));
+		}
+
+		assertEquals(1, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size());
+		assertEquals(expected[2], checkpoints.getLatestCheckpoint());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/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 5a5ef57..fa61acf 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
@@ -178,6 +178,10 @@ public class BlobLibraryCacheManagerTest {
 
 				// un-register them again
 				libCache.unregisterTask(jid, executionId);
+
+				// Don't fail if called again
+				libCache.unregisterTask(jid, executionId);
+
 				assertEquals(0, libCache.getNumberOfReferenceHolders(jid));
 
 				// library is still cached (but not associated with job any more)

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
index 56e5bde..ca8810b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
@@ -25,7 +25,7 @@ import io.netty.channel.ChannelPromise;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.ConnectionID;
-import org.apache.flink.runtime.net.NetUtils;
+import org.apache.flink.util.NetUtils;
 import org.junit.Ignore;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
new file mode 100644
index 0000000..ac250bd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.Props;
+import akka.actor.UntypedActor;
+import akka.testkit.TestActorRef;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.leaderelection.TestingListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.JobManagerMessages.JobStatusResponse;
+import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage;
+import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
+import org.apache.flink.runtime.testutils.JobManagerProcess;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.util.TestLogger;
+import org.apache.zookeeper.data.Stat;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests recovery of {@link SubmittedJobGraph} instances.
+ */
+public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	private static final File FileStateBackendBasePath;
+
+	static {
+		try {
+			FileStateBackendBasePath = CommonTestUtils.createTempDirectory();
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Error in test setup. Could not create directory.", e);
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		ZooKeeper.shutdown();
+
+		if (FileStateBackendBasePath != null) {
+			FileUtils.deleteDirectory(FileStateBackendBasePath);
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		if (FileStateBackendBasePath != null) {
+			FileUtils.cleanDirectory(FileStateBackendBasePath);
+		}
+
+		ZooKeeper.deleteAll();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Tests that the recovery state is cleaned up after a JobManager stops.
+	 */
+	@Test
+	public void testJobManagerCleanUp() throws Exception {
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+				ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());
+
+		// Configure the cluster
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 1);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+
+		TestingCluster flink = new TestingCluster(config, false, false, StreamingMode.STREAMING);
+
+		try {
+			final Deadline deadline = TestTimeOut.fromNow();
+
+			// Start the JobManager and TaskManager
+			flink.start(true);
+
+			JobGraph jobGraph = createBlockingJobGraph();
+
+			ActorGateway jobManager = flink.getLeaderGateway(deadline.timeLeft());
+
+			// Submit the job
+			jobManager.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED));
+
+			// Wait for the job to start
+			JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING,
+					jobManager, deadline.timeLeft());
+		}
+		finally {
+			flink.shutdown();
+		}
+
+		// Verify that everything is clean
+		verifyCleanRecoveryState(config);
+	}
+
+	/**
+	 * Tests that submissions to non-leaders are handled.
+	 */
+	@Test
+	public void testSubmitJobToNonLeader() throws Exception {
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+				ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());
+
+		// Configure the cluster
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 2);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+
+		TestingCluster flink = new TestingCluster(config, false, false, StreamingMode.STREAMING);
+
+		try {
+			final Deadline deadline = TestTimeOut.fromNow();
+
+			// Start the JobManager and TaskManager
+			flink.start(true);
+
+			JobGraph jobGraph = createBlockingJobGraph();
+
+			List<ActorRef> bothJobManagers = flink.getJobManagersAsJava();
+
+			ActorGateway leadingJobManager = flink.getLeaderGateway(deadline.timeLeft());
+
+			ActorGateway nonLeadingJobManager;
+			if (bothJobManagers.get(0).equals(leadingJobManager.actor())) {
+				nonLeadingJobManager = new AkkaActorGateway(bothJobManagers.get(1), null);
+			}
+			else {
+				nonLeadingJobManager = new AkkaActorGateway(bothJobManagers.get(0), null);
+			}
+
+			// Submit the job
+			nonLeadingJobManager.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED));
+
+			// Wait for the job to start. We are asking the *leading** JM here although we've
+			// submitted the job to the non-leading JM. This is the behaviour under test.
+			JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING,
+					leadingJobManager, deadline.timeLeft());
+
+			// Make sure that the **non-leading** JM has actually removed the job graph from her
+			// local state.
+			boolean success = false;
+			while (!success && deadline.hasTimeLeft()) {
+				JobStatusResponse jobStatusResponse = JobManagerActorTestUtils.requestJobStatus(
+						jobGraph.getJobID(), nonLeadingJobManager, deadline.timeLeft());
+
+				if (jobStatusResponse instanceof JobManagerMessages.JobNotFound) {
+					success = true;
+				}
+				else {
+					Thread.sleep(100);
+				}
+			}
+
+			if (!success) {
+				fail("Non-leading JM was still holding reference to the job graph.");
+			}
+		}
+		finally {
+			flink.shutdown();
+		}
+
+		// Verify that everything is clean
+		verifyCleanRecoveryState(config);
+	}
+
+	/**
+	 * Tests that clients receive updates after recovery by a new leader.
+	 */
+	@Test
+	public void testClientNonDetachedListeningBehaviour() throws Exception {
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+				ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());
+
+		// Test actor system
+		ActorSystem testSystem = null;
+
+		// JobManager setup. Start the job managers as separate processes in order to not run the
+		// actors postStop, which cleans up all running jobs.
+		JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2];
+
+		LeaderRetrievalService leaderRetrievalService = null;
+
+		ActorSystem taskManagerSystem = null;
+
+		try {
+			final Deadline deadline = TestTimeOut.fromNow();
+
+			// Test actor system
+			testSystem = AkkaUtils.createActorSystem(new Configuration(),
+					new Some<>(new Tuple2<String, Object>("localhost", 0)));
+
+			// The job managers
+			jobManagerProcess[0] = new JobManagerProcess(0, config);
+			jobManagerProcess[1] = new JobManagerProcess(1, config);
+
+			jobManagerProcess[0].createAndStart();
+			jobManagerProcess[1].createAndStart();
+
+			// Leader listener
+			TestingListener leaderListener = new TestingListener();
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService.start(leaderListener);
+
+			// The task manager
+			taskManagerSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+			TaskManager.startTaskManagerComponentsAndActor(
+					config, taskManagerSystem, "localhost",
+					Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
+					false, StreamingMode.STREAMING, TaskManager.class);
+
+			// Client test actor
+			TestActorRef<RecordingTestClient> clientRef = TestActorRef.create(
+					testSystem, Props.create(RecordingTestClient.class));
+
+			JobGraph jobGraph = createBlockingJobGraph();
+
+			{
+				// Initial submission
+				leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+				String leaderAddress = leaderListener.getAddress();
+				UUID leaderId = leaderListener.getLeaderSessionID();
+
+				// The client
+				AkkaActorGateway client = new AkkaActorGateway(clientRef, leaderId);
+
+				// Get the leader ref
+				ActorRef leaderRef = AkkaUtils.getActorRef(
+						leaderAddress, testSystem, deadline.timeLeft());
+				ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId);
+
+				// Submit the job in non-detached mode
+				leader.tell(new SubmitJob(jobGraph,
+						ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES), client);
+
+				JobManagerActorTestUtils.waitForJobStatus(
+						jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft());
+			}
+
+			// Who's the boss?
+			JobManagerProcess leadingJobManagerProcess;
+			if (jobManagerProcess[0].getJobManagerAkkaURL().equals(leaderListener.getAddress())) {
+				leadingJobManagerProcess = jobManagerProcess[0];
+			}
+			else {
+				leadingJobManagerProcess = jobManagerProcess[1];
+			}
+
+			// Kill the leading job manager process
+			leadingJobManagerProcess.destroy();
+
+			{
+				// Recovery by the standby JobManager
+				leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());
+
+				String leaderAddress = leaderListener.getAddress();
+				UUID leaderId = leaderListener.getLeaderSessionID();
+
+				ActorRef leaderRef = AkkaUtils.getActorRef(
+						leaderAddress, testSystem, deadline.timeLeft());
+				ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId);
+
+				JobManagerActorTestUtils.waitForJobStatus(
+						jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft());
+
+				// Cancel the job
+				leader.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
+			}
+
+			// Wait for the execution result
+			clientRef.underlyingActor().awaitJobResult(deadline.timeLeft().toMillis());
+
+			int jobSubmitSuccessMessages = 0;
+			for (Object msg : clientRef.underlyingActor().getMessages()) {
+				if (msg instanceof JobManagerMessages.JobSubmitSuccess) {
+					jobSubmitSuccessMessages++;
+				}
+			}
+
+			// At least two submissions should be ack-ed (initial and recovery). This is quite
+			// conservative, but it is still possible that these messages are overtaken by the
+			// final message.
+			assertEquals(2, jobSubmitSuccessMessages);
+		}
+		catch (Throwable t) {
+			// In case of an error, print the job manager process logs.
+			if (jobManagerProcess[0] != null) {
+				jobManagerProcess[0].printProcessLog();
+			}
+
+			if (jobManagerProcess[1] != null) {
+				jobManagerProcess[1].printProcessLog();
+			}
+
+			t.printStackTrace();
+		}
+		finally {
+			if (jobManagerProcess[0] != null) {
+				jobManagerProcess[0].destroy();
+			}
+
+			if (jobManagerProcess[1] != null) {
+				jobManagerProcess[1].destroy();
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			if (taskManagerSystem != null) {
+				taskManagerSystem.shutdown();
+			}
+
+			if (testSystem != null) {
+				testSystem.shutdown();
+			}
+		}
+	}
+
+	/**
+	 * Simple recording client.
+	 */
+	private static class RecordingTestClient extends UntypedActor {
+
+		private final Queue<Object> messages = new ConcurrentLinkedQueue<>();
+
+		private CountDownLatch jobResultLatch = new CountDownLatch(1);
+
+		@Override
+		public void onReceive(Object message) throws Exception {
+			if (message instanceof LeaderSessionMessage) {
+				message = ((LeaderSessionMessage) message).message();
+			}
+
+			messages.add(message);
+
+			// Check for job result
+			if (message instanceof JobManagerMessages.JobResultFailure ||
+					message instanceof JobManagerMessages.JobResultSuccess) {
+
+				jobResultLatch.countDown();
+			}
+		}
+
+		public Queue<Object> getMessages() {
+			return messages;
+		}
+
+		public void awaitJobResult(long timeout) throws InterruptedException {
+			jobResultLatch.await(timeout, TimeUnit.MILLISECONDS);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates a simple blocking JobGraph.
+	 */
+	private static JobGraph createBlockingJobGraph() {
+		JobGraph jobGraph = new JobGraph("Blocking program");
+
+		JobVertex jobVertex = new JobVertex("Blocking Vertex");
+		jobVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+
+		jobGraph.addVertex(jobVertex);
+
+		return jobGraph;
+	}
+
+	/**
+	 * Fails the test if the recovery state (file state backend and ZooKeeper) is not clean.
+	 */
+	private static void verifyCleanRecoveryState(Configuration config) throws Exception {
+		// File state backend empty
+		Collection<File> stateHandles = FileUtils.listFiles(
+				FileStateBackendBasePath, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE);
+
+		if (!stateHandles.isEmpty()) {
+			fail("File state backend is not clean: " + stateHandles);
+		}
+
+		// ZooKeeper
+		String currentJobsPath = config.getString(
+				ConfigConstants.ZOOKEEPER_JOBGRAPHS_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
+
+		Stat stat = ZooKeeper.getClient().checkExists().forPath(currentJobsPath);
+
+		if (stat.getCversion() == 0) {
+			// Sanity check: verify that some changes have been performed
+			fail("ZooKeeper state for '" + currentJobsPath + "' has not been modified during " +
+					"this test. What are you testing?");
+		}
+
+		if (stat.getNumChildren() != 0) {
+			// Is everything clean again?
+			fail("ZooKeeper path '" + currentJobsPath + "' is not clean: " +
+					ZooKeeper.getClient().getChildren().forPath(currentJobsPath));
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java
new file mode 100644
index 0000000..753e7be
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import akka.actor.ActorRef;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class StandaloneSubmittedJobGraphStoreTest {
+
+	/**
+	 * Tests that all operations work and don't change the state.
+	 */
+	@Test
+	public void testNoOps() throws Exception {
+		StandaloneSubmittedJobGraphStore jobGraphs = new StandaloneSubmittedJobGraphStore();
+
+		SubmittedJobGraph jobGraph = new SubmittedJobGraph(
+				new JobGraph("testNoOps"),
+				new JobInfo(ActorRef.noSender(), ListeningBehaviour.DETACHED, 0, Integer.MAX_VALUE));
+
+		assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+		jobGraphs.putJobGraph(jobGraph);
+		assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+		jobGraphs.removeJobGraph(jobGraph.getJobGraph().getJobID());
+		assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+		assertTrue(jobGraphs.recoverJobGraph(new JobID()).isEmpty());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
new file mode 100644
index 0000000..861a713
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import akka.actor.ActorRef;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener;
+import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider;
+import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.atMost;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for basic {@link SubmittedJobGraphStore} contract.
+ */
+public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	private final static LocalStateHandleProvider<SubmittedJobGraph> StateHandleProvider =
+			new LocalStateHandleProvider<>();
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (ZooKeeper != null) {
+			ZooKeeper.shutdown();
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		ZooKeeper.deleteAll();
+	}
+
+	@Test
+	public void testPutAndRemoveJobGraph() throws Exception {
+		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
+				ZooKeeper.createClient(), "/testPutAndRemoveJobGraph",
+				StateHandleProvider);
+
+		try {
+			SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class);
+
+			jobGraphs.start(listener);
+
+			SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0);
+
+			// Empty state
+			assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+			// Add initial
+			jobGraphs.putJobGraph(jobGraph);
+
+			// Verify initial job graph
+			List<SubmittedJobGraph> actual = jobGraphs.recoverJobGraphs();
+			assertEquals(1, actual.size());
+
+			verifyJobGraphs(jobGraph, actual.get(0));
+
+			// Update (same ID)
+			jobGraph = createSubmittedJobGraph(jobGraph.getJobId(), 1);
+			jobGraphs.putJobGraph(jobGraph);
+
+			// Verify updated
+			actual = jobGraphs.recoverJobGraphs();
+			assertEquals(1, actual.size());
+
+			verifyJobGraphs(jobGraph, actual.get(0));
+
+			// Remove
+			jobGraphs.removeJobGraph(jobGraph.getJobId());
+
+			// Empty state
+			assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+			// Nothing should have been notified
+			verify(listener, atMost(1)).onAddedJobGraph(any(JobID.class));
+			verify(listener, never()).onRemovedJobGraph(any(JobID.class));
+
+			// Don't fail if called again
+			jobGraphs.removeJobGraph(jobGraph.getJobId());
+		}
+		finally {
+			jobGraphs.stop();
+		}
+	}
+
+	@Test
+	public void testRecoverJobGraphs() throws Exception {
+		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
+				ZooKeeper.createClient(), "/testRecoverJobGraphs", StateHandleProvider);
+
+		try {
+			SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class);
+
+			jobGraphs.start(listener);
+
+			HashMap<JobID, SubmittedJobGraph> expected = new HashMap<>();
+			JobID[] jobIds = new JobID[] { new JobID(), new JobID(), new JobID() };
+
+			expected.put(jobIds[0], createSubmittedJobGraph(jobIds[0], 0));
+			expected.put(jobIds[1], createSubmittedJobGraph(jobIds[1], 1));
+			expected.put(jobIds[2], createSubmittedJobGraph(jobIds[2], 2));
+
+			// Add all
+			for (SubmittedJobGraph jobGraph : expected.values()) {
+				jobGraphs.putJobGraph(jobGraph);
+			}
+
+			List<SubmittedJobGraph> actual = jobGraphs.recoverJobGraphs();
+
+			assertEquals(expected.size(), actual.size());
+
+			for (SubmittedJobGraph jobGraph : actual) {
+				assertTrue(expected.containsKey(jobGraph.getJobId()));
+
+				verifyJobGraphs(expected.get(jobGraph.getJobId()), jobGraph);
+
+				jobGraphs.removeJobGraph(jobGraph.getJobId());
+			}
+
+			// Empty state
+			assertEquals(0, jobGraphs.recoverJobGraphs().size());
+
+			// Nothing should have been notified
+			verify(listener, atMost(expected.size())).onAddedJobGraph(any(JobID.class));
+			verify(listener, never()).onRemovedJobGraph(any(JobID.class));
+		}
+		finally {
+			jobGraphs.stop();
+		}
+	}
+
+	@Test
+	public void testConcurrentAddJobGraph() throws Exception {
+		ZooKeeperSubmittedJobGraphStore jobGraphs = null;
+		ZooKeeperSubmittedJobGraphStore otherJobGraphs = null;
+
+		try {
+			jobGraphs = new ZooKeeperSubmittedJobGraphStore(
+					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider);
+
+			otherJobGraphs = new ZooKeeperSubmittedJobGraphStore(
+					ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider);
+
+
+			SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0);
+			SubmittedJobGraph otherJobGraph = createSubmittedJobGraph(new JobID(), 0);
+
+			SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class);
+
+			final JobID[] actualOtherJobId = new JobID[1];
+			final CountDownLatch sync = new CountDownLatch(1);
+
+			doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) throws Throwable {
+					actualOtherJobId[0] = (JobID) invocation.getArguments()[0];
+					sync.countDown();
+
+					return null;
+				}
+			}).when(listener).onAddedJobGraph(any(JobID.class));
+
+			// Test
+			jobGraphs.start(listener);
+			otherJobGraphs.start(null);
+
+			jobGraphs.putJobGraph(jobGraph);
+
+			// Everything is cool... not much happening ;)
+			verify(listener, never()).onAddedJobGraph(any(JobID.class));
+			verify(listener, never()).onRemovedJobGraph(any(JobID.class));
+
+			// This bad boy adds the other job graph
+			otherJobGraphs.putJobGraph(otherJobGraph);
+
+			// Wait for the cache to call back
+			sync.await();
+
+			verify(listener, times(1)).onAddedJobGraph(any(JobID.class));
+			verify(listener, never()).onRemovedJobGraph(any(JobID.class));
+
+			assertEquals(otherJobGraph.getJobId(), actualOtherJobId[0]);
+		}
+		finally {
+			if (jobGraphs != null) {
+				jobGraphs.stop();
+			}
+
+			if (otherJobGraphs != null) {
+				otherJobGraphs.stop();
+			}
+		}
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testUpdateJobGraphYouDidNotGetOrAdd() throws Exception {
+		ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore(
+				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider);
+
+		ZooKeeperSubmittedJobGraphStore otherJobGraphs = new ZooKeeperSubmittedJobGraphStore(
+				ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider);
+
+		jobGraphs.start(null);
+		otherJobGraphs.start(null);
+
+		SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0);
+
+		jobGraphs.putJobGraph(jobGraph);
+
+		otherJobGraphs.putJobGraph(jobGraph);
+	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	private SubmittedJobGraph createSubmittedJobGraph(JobID jobId, long start) {
+		final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph");
+
+		final JobVertex jobVertex = new JobVertex("Test JobVertex");
+		jobVertex.setParallelism(1);
+
+		jobGraph.addVertex(jobVertex);
+
+		final JobInfo jobInfo = new JobInfo(
+				ActorRef.noSender(), ListeningBehaviour.DETACHED, start, Integer.MAX_VALUE);
+
+		return new SubmittedJobGraph(jobGraph, jobInfo);
+	}
+
+	protected void verifyJobGraphs(SubmittedJobGraph expected, SubmittedJobGraph actual)
+			throws Exception {
+
+		JobGraph expectedJobGraph = expected.getJobGraph();
+		JobGraph actualJobGraph = actual.getJobGraph();
+
+		assertEquals(expectedJobGraph.getName(), actualJobGraph.getName());
+		assertEquals(expectedJobGraph.getJobID(), actualJobGraph.getJobID());
+
+		JobInfo expectedJobInfo = expected.getJobInfo();
+		JobInfo actualJobInfo = actual.getJobInfo();
+
+		assertEquals(expectedJobInfo.listeningBehaviour(), actualJobInfo.listeningBehaviour());
+		assertEquals(expectedJobInfo.start(), actualJobInfo.start());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
index 753bbab..bbd8fad 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
@@ -25,19 +25,25 @@ import akka.actor.Props;
 import akka.pattern.Patterns;
 import akka.testkit.JavaTestKit;
 import akka.util.Timeout;
+import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.instance.InstanceManager;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.testingUtils.TestingJobManager;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.util.LeaderElectionUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -151,8 +157,19 @@ public class JobManagerLeaderElectionTest extends TestLogger {
 	}
 
 	private Props createJobManagerProps(Configuration configuration) throws Exception {
-		LeaderElectionService leaderElectionService = LeaderElectionUtils.
-				createLeaderElectionService(configuration);
+		LeaderElectionService leaderElectionService;
+		if (RecoveryMode.fromConfig(configuration) == RecoveryMode.STANDALONE) {
+			leaderElectionService = new StandaloneLeaderElectionService();
+		}
+		else {
+			CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration);
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client,
+					configuration);
+		}
+
+		// We don't need recovery in this test
+		SubmittedJobGraphStore submittedJobGraphStore = new StandaloneSubmittedJobGraphStore();
+		CheckpointRecoveryFactory checkpointRecoveryFactory = new StandaloneCheckpointRecoveryFactory();
 
 		return Props.create(
 				TestingJobManager.class,
@@ -166,7 +183,9 @@ public class JobManagerLeaderElectionTest extends TestLogger {
 				1L,
 				AkkaUtils.getDefaultTimeout(),
 				StreamingMode.BATCH_ONLY,
-				leaderElectionService
+				leaderElectionService,
+				submittedJobGraphStore,
+				checkpointRecoveryFactory
 		);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
index c4fccd7..ea058f4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
@@ -27,6 +27,8 @@ import java.util.UUID;
  */
 public class TestingLeaderElectionService implements LeaderElectionService, Serializable {
 
+	private static final long serialVersionUID = -8007939683948014574L;
+
 	private LeaderContender contender;
 	private boolean hasLeadership = false;
 
@@ -51,10 +53,12 @@ public class TestingLeaderElectionService implements LeaderElectionService, Seri
 	}
 
 	public void isLeader(UUID leaderSessionID) {
+		hasLeadership = true;
 		contender.grantLeadership(leaderSessionID);
 	}
 
 	public void notLeader() {
+		hasLeadership = false;
 		contender.revokeLeadership();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
index bb60415..aae1840 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
@@ -18,13 +18,14 @@
 
 package org.apache.flink.runtime.leaderelection;
 
+import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.util.LeaderElectionUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
@@ -41,7 +42,7 @@ import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
@@ -92,7 +93,11 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 		Thread thread;
 
+		CuratorFramework[] client = new CuratorFramework[2];
+
 		try {
+			client[0] = ZooKeeperUtils.startCuratorFramework(config);
+			client[1] = ZooKeeperUtils.startCuratorFramework(config);
 
 			InetSocketAddress wrongInetSocketAddress = new InetSocketAddress(InetAddress.getByName("1.1.1.1"), 1234);
 
@@ -116,7 +121,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 			String correctAddress = JobManager.getRemoteJobManagerAkkaURL(correctInetSocketAddress, Option.<String>empty());
 
-			faultyLeaderElectionService = LeaderElectionUtils.createLeaderElectionService(config);
+			faultyLeaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[0], config);
 			TestingContender wrongLeaderAddressContender = new TestingContender(wrongAddress, faultyLeaderElectionService);
 
 			faultyLeaderElectionService.start(wrongLeaderAddressContender);
@@ -127,7 +132,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 			thread.start();
 
-			leaderElectionService = LeaderElectionUtils.createLeaderElectionService(config);
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[1], config);
 			TestingContender correctLeaderAddressContender = new TestingContender(correctAddress, leaderElectionService);
 
 			Thread.sleep(sleepingTime);
@@ -155,6 +160,14 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 			if (leaderElectionService != null) {
 				leaderElectionService.stop();
 			}
+
+			if (client[0] != null) {
+				client[0].close();
+			}
+
+			if (client[1] != null) {
+				client[1].close();
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
index 68575e5..087e0fd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
@@ -78,7 +78,7 @@ public class CheckpointMessagesTest {
 		assertNotNull(copy.toString());
 	}
 	
-	private static class MyHandle implements StateHandle<Serializable> {
+	public static class MyHandle implements StateHandle<Serializable> {
 
 		private static final long serialVersionUID = 8128146204128728332L;
 


[37/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot
new file mode 100644
index 0000000..33b2bb8
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.eot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
new file mode 100644
index 0000000..1ee89d4
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.svg
@@ -0,0 +1,565 @@
+<?xml version="1.0" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
+<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1">
+<metadata></metadata>
+<defs>
+<font id="fontawesomeregular" horiz-adv-x="1536" >
+<font-face units-per-em="1792" ascent="1536" descent="-256" />
+<missing-glyph horiz-adv-x="448" />
+<glyph unicode=" "  horiz-adv-x="448" />
+<glyph unicode="&#x09;" horiz-adv-x="448" />
+<glyph unicode="&#xa0;" horiz-adv-x="448" />
+<glyph unicode="&#xa8;" horiz-adv-x="1792" />
+<glyph unicode="&#xa9;" horiz-adv-x="1792" />
+<glyph unicode="&#xae;" horiz-adv-x="1792" />
+<glyph unicode="&#xb4;" horiz-adv-x="1792" />
+<glyph unicode="&#xc6;" horiz-adv-x="1792" />
+<glyph unicode="&#xd8;" horiz-adv-x="1792" />
+<glyph unicode="&#x2000;" horiz-adv-x="768" />
+<glyph unicode="&#x2001;" horiz-adv-x="1537" />
+<glyph unicode="&#x2002;" horiz-adv-x="768" />
+<glyph unicode="&#x2003;" horiz-adv-x="1537" />
+<glyph unicode="&#x2004;" horiz-adv-x="512" />
+<glyph unicode="&#x2005;" horiz-adv-x="384" />
+<glyph unicode="&#x2006;" horiz-adv-x="256" />
+<glyph unicode="&#x2007;" horiz-adv-x="256" />
+<glyph unicode="&#x2008;" horiz-adv-x="192" />
+<glyph unicode="&#x2009;" horiz-adv-x="307" />
+<glyph unicode="&#x200a;" horiz-adv-x="85" />
+<glyph unicode="&#x202f;" horiz-adv-x="307" />
+<glyph unicode="&#x205f;" horiz-adv-x="384" />
+<glyph unicode="&#x2122;" horiz-adv-x="1792" />
+<glyph unicode="&#x221e;" horiz-adv-x="1792" />
+<glyph unicode="&#x2260;" horiz-adv-x="1792" />
+<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
+<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
+<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
+<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
+<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
+<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
+<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
+<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
+<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
+<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
+<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
+<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
+<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
+<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
+<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
+<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
+<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
+<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
+<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
+<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
+<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
+<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
+<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
+<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
+<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
+<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
+<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
+<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
+<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
+<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
+<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
+<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
+<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
+<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
+<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
+<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
+<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
+<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
+<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
+<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
+<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
+<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
+<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
+<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
+<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
+<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
+<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
+<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
+<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
+<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
+<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
+<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
+<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
+<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
+<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
+<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
+<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
+<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
+<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
+<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
+<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
+<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
+<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
+<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
+<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
+<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
+<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
+<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf082;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 q0 119 84.5 203.5t203.5 84.5h960z" />
+<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
+<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
+<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
+<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
+<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
+<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
+<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
+<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
+<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
+<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
+<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
+<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
+<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
+<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
+<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
+<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
+<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
+<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
+<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
+<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
+<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
+<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
+<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
+<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
+<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
+<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
+<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11.5 -8.5t-9 -6.5t-8.5

<TRUNCATED>

[47/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
[FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

The old StateHandleProvider used in ZooKeeperStateHandleStore had to be replaced because the state backend implementation has changed. Since the new state backend could not be used anymore, a new StateStorageHelper interface has been created. The default implementation FileSystemStateStorageHelper stores the given state onto the specified file system and returns a FileSerializableStateHandle.

Various fixes due to rebasing.


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

Branch: refs/heads/master
Commit: a6890b2849db34741dbae16fd144d5ec98cc51d8
Parents: 630798d
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 19 11:30:02 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 09:57:34 2015 +0200

----------------------------------------------------------------------
 .../runtime/webmonitor/WebRuntimeMonitor.java   |     4 +-
 .../files/StaticFileServerHandler.java          |     4 +-
 .../src/main/resources/web/css/index.css        |   596 -
 .../src/main/resources/web/css/vendor.css       |  9183 --
 .../main/resources/web/fonts/FontAwesome.otf    |   Bin 93888 -> 0 bytes
 .../resources/web/fonts/fontawesome-webfont.eot |   Bin 60767 -> 0 bytes
 .../resources/web/fonts/fontawesome-webfont.svg |   565 -
 .../resources/web/fonts/fontawesome-webfont.ttf |   Bin 122092 -> 0 bytes
 .../web/fonts/fontawesome-webfont.woff          |   Bin 71508 -> 0 bytes
 .../web/fonts/fontawesome-webfont.woff2         |   Bin 56780 -> 0 bytes
 .../main/resources/web/images/flink-logo.png    |   Bin 6096 -> 0 bytes
 .../src/main/resources/web/index.html           |    55 -
 .../src/main/resources/web/js/index.js          |  1420 -
 .../src/main/resources/web/js/index.js.orig     |  1150 -
 .../src/main/resources/web/js/vendor.js         | 81087 -----------------
 .../web/partials/jobmanager/config.html         |    33 -
 .../web/partials/jobmanager/index.html          |    33 -
 .../web/partials/jobmanager/stdout.html         |    40 -
 .../web/partials/jobs/completed-jobs.html       |    53 -
 .../resources/web/partials/jobs/job.config.html |    57 -
 .../web/partials/jobs/job.exceptions.html       |    38 -
 .../main/resources/web/partials/jobs/job.html   |    48 -
 .../resources/web/partials/jobs/job.plan.html   |    31 -
 .../jobs/job.plan.node-list.accumulators.html   |    40 -
 .../jobs/job.plan.node-list.overview.html       |    60 -
 .../jobs/job.plan.node.accumulators.html        |    68 -
 .../partials/jobs/job.plan.node.subtasks.html   |    52 -
 .../web/partials/jobs/job.properties.html       |   140 -
 .../web/partials/jobs/job.statistics.html       |    40 -
 .../web/partials/jobs/job.timeline.html         |    23 -
 .../web/partials/jobs/job.timeline.vertex.html  |    30 -
 .../web/partials/jobs/running-jobs.html         |    53 -
 .../main/resources/web/partials/overview.html   |   147 -
 .../webmonitor/WebRuntimeMonitorITCase.java     |    25 +-
 flink-runtime-web/web-dashboard/gulpfile.js     |     4 +-
 .../web-dashboard/web/css/index.css             |   596 +
 .../web-dashboard/web/css/vendor.css            |  9183 ++
 .../web-dashboard/web/fonts/FontAwesome.otf     |   Bin 0 -> 93888 bytes
 .../web/fonts/fontawesome-webfont.eot           |   Bin 0 -> 60767 bytes
 .../web/fonts/fontawesome-webfont.svg           |   565 +
 .../web/fonts/fontawesome-webfont.ttf           |   Bin 0 -> 122092 bytes
 .../web/fonts/fontawesome-webfont.woff          |   Bin 0 -> 71508 bytes
 .../web/fonts/fontawesome-webfont.woff2         |   Bin 0 -> 56780 bytes
 .../web-dashboard/web/images/flink-logo.png     |   Bin 0 -> 6096 bytes
 flink-runtime-web/web-dashboard/web/index.html  |    55 +
 flink-runtime-web/web-dashboard/web/js/index.js |  1420 +
 .../web-dashboard/web/js/vendor.js              | 81087 +++++++++++++++++
 .../web/partials/jobmanager/config.html         |    33 +
 .../web/partials/jobmanager/index.html          |    33 +
 .../web/partials/jobmanager/stdout.html         |    40 +
 .../web/partials/jobs/completed-jobs.html       |    53 +
 .../web/partials/jobs/job.config.html           |    57 +
 .../web/partials/jobs/job.exceptions.html       |    38 +
 .../web-dashboard/web/partials/jobs/job.html    |    48 +
 .../web/partials/jobs/job.plan.html             |    31 +
 .../jobs/job.plan.node-list.accumulators.html   |    40 +
 .../jobs/job.plan.node-list.overview.html       |    60 +
 .../jobs/job.plan.node.accumulators.html        |    68 +
 .../partials/jobs/job.plan.node.subtasks.html   |    52 +
 .../web/partials/jobs/job.properties.html       |   140 +
 .../web/partials/jobs/job.statistics.html       |    40 +
 .../web/partials/jobs/job.timeline.html         |    23 +
 .../web/partials/jobs/job.timeline.vertex.html  |    30 +
 .../web/partials/jobs/running-jobs.html         |    53 +
 .../web-dashboard/web/partials/overview.html    |   147 +
 .../web/partials/taskmanager/index.html         |    57 +
 .../flink/runtime/blob/FileSystemBlobStore.java |    44 +-
 .../ZooKeeperCompletedCheckpointStore.java      |    12 +-
 .../ZooKeeperSubmittedJobGraphStore.java        |    20 +-
 .../runtime/state/AbstractHeapKvState.java      |   145 +
 .../org/apache/flink/runtime/state/KvState.java |    69 +
 .../flink/runtime/state/KvStateSnapshot.java    |    69 +
 .../runtime/state/SerializedCheckpointData.java |   176 +
 .../flink/runtime/state/StateBackend.java       |   200 +-
 .../runtime/state/StateBackendFactory.java      |    40 +
 .../state/StateHandleProviderFactory.java       |    61 -
 .../flink/runtime/state/StreamStateHandle.java  |    28 +
 .../state/filesystem/AbstractFileState.java     |    83 +
 .../filesystem/FileSerializableStateHandle.java |    53 +
 .../state/filesystem/FileStreamStateHandle.java |    46 +
 .../runtime/state/filesystem/FsHeapKvState.java |    88 +
 .../state/filesystem/FsHeapKvStateSnapshot.java |    95 +
 .../state/filesystem/FsStateBackend.java        |   412 +
 .../state/filesystem/FsStateBackendFactory.java |    56 +
 .../state/memory/ByteStreamStateHandle.java     |    52 +
 .../runtime/state/memory/MemHeapKvState.java    |    52 +
 .../state/memory/MemoryHeapKvStateSnapshot.java |   102 +
 .../state/memory/MemoryStateBackend.java        |   209 +
 .../state/memory/SerializedStateHandle.java     |    49 +
 .../flink/runtime/util/ZooKeeperUtils.java      |    56 +-
 .../runtime/zookeeper/StateStorageHelper.java   |    41 +
 .../zookeeper/ZooKeeperStateHandleStore.java    |    26 +-
 .../FileSystemStateStorageHelper.java           |    86 +
 .../flink/runtime/jobmanager/JobManager.scala   |    43 +-
 ...ZooKeeperCompletedCheckpointStoreITCase.java |    10 +-
 .../BlobLibraryCacheRecoveryITCase.java         |    28 +-
 .../ZooKeeperSubmittedJobGraphsStoreITCase.java |    28 +-
 .../runtime/state/FileStateBackendTest.java     |   423 +
 .../runtime/state/MemoryStateBackendTest.java   |   282 +
 .../runtime/testutils/ZooKeeperTestUtils.java   |     3 +-
 .../ZooKeeperStateHandleStoreITCase.java        |    60 +-
 .../src/test/resources/log4j-test.properties    |     3 +-
 .../flink/hdfstests/FileStateBackendTest.java   |     8 +-
 .../environment/StreamExecutionEnvironment.java |     6 +-
 .../source/MessageAcknowledingSourceBase.java   |     3 +-
 .../flink/streaming/api/graph/StreamConfig.java |     2 +-
 .../flink/streaming/api/graph/StreamGraph.java  |     2 +-
 .../api/operators/AbstractStreamOperator.java   |     6 +-
 .../operators/AbstractUdfStreamOperator.java    |     2 +-
 .../api/state/AbstractHeapKvState.java          |   145 -
 .../flink/streaming/api/state/KvState.java      |    69 -
 .../streaming/api/state/KvStateSnapshot.java    |    69 -
 .../api/state/SerializedCheckpointData.java     |   176 -
 .../flink/streaming/api/state/StateBackend.java |   214 -
 .../api/state/StateBackendFactory.java          |    40 -
 .../streaming/api/state/StreamStateHandle.java  |    28 -
 .../api/state/filesystem/AbstractFileState.java |    83 -
 .../filesystem/FileSerializableStateHandle.java |    53 -
 .../state/filesystem/FileStreamStateHandle.java |    46 -
 .../api/state/filesystem/FsHeapKvState.java     |    88 -
 .../state/filesystem/FsHeapKvStateSnapshot.java |    95 -
 .../api/state/filesystem/FsStateBackend.java    |   412 -
 .../state/filesystem/FsStateBackendFactory.java |    56 -
 .../api/state/memory/ByteStreamStateHandle.java |    52 -
 .../api/state/memory/MemHeapKvState.java        |    52 -
 .../state/memory/MemoryHeapKvStateSnapshot.java |   102 -
 .../api/state/memory/MemoryStateBackend.java    |   209 -
 .../api/state/memory/SerializedStateHandle.java |    49 -
 ...ractAlignedProcessingTimeWindowOperator.java |     2 +-
 .../streaming/runtime/tasks/StreamTask.java     |    42 +-
 .../runtime/tasks/StreamTaskState.java          |     2 +-
 .../api/state/FileStateBackendTest.java         |   419 -
 .../api/state/MemoryStateBackendTest.java       |   278 -
 ...AlignedProcessingTimeWindowOperatorTest.java |     4 +-
 ...AlignedProcessingTimeWindowOperatorTest.java |     4 +-
 .../flink/streaming/util/MockContext.java       |     4 +-
 .../util/OneInputStreamOperatorTestHarness.java |     4 +-
 .../util/TwoInputStreamOperatorTestHarness.java |     4 +-
 .../api/scala/StreamExecutionEnvironment.scala  |     2 +-
 .../test/classloading/ClassLoaderITCase.java    |     2 +-
 .../flink/test/recovery/ChaosMonkeyITCase.java  |     3 +-
 .../JobManagerCheckpointRecoveryITCase.java     |     5 +-
 ...erProcessFailureStreamingRecoveryITCase.java |     2 +-
 .../ZooKeeperLeaderElectionITCase.java          |     4 +-
 .../src/test/resources/log4j-test.properties    |     1 -
 .../flink/yarn/YARNHighAvailabilityITCase.java  |     3 +-
 .../flink/yarn/ApplicationMasterBase.scala      |     2 +-
 pom.xml                                         |     2 +-
 148 files changed, 97046 insertions(+), 98088 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index ec973c7..40d9f2d 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -192,8 +192,6 @@ public class WebRuntimeMonitor implements WebMonitor {
 
 			// job manager configuration, log and stdout
 			.GET("/jobmanager/config", handler(new JobManagerConfigHandler(config)))
-			.GET("/jobmanager/log", new StaticFileServerHandler(logDir))
-			.GET("/jobmanager/stdout", new StaticFileServerHandler(outDir))
 
 			// overview over jobs
 			.GET("/joboverview", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true)))
@@ -222,6 +220,8 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT)))
 			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT)))
 
+			.GET("/jobmanager/log", new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logDir))
+			.GET("/jobmanager/stdout", new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, outDir))
 			// this handler serves all the static contents
 			.GET("/:*", new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, webRootDir));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
index d46a900..02dd81e 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
@@ -58,10 +58,12 @@ import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.FilenameFilter;
 import java.io.RandomAccessFile;
 import java.nio.file.Files;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
 import java.util.Date;
@@ -198,7 +200,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 	 * Response when running with leading JobManager.
 	 */
 	private void respondAsLeader(ChannelHandlerContext ctx, HttpRequest request, String requestPath)
-			throws ParseException, IOException {
+		throws IOException, ParseException {
 
 		// convert to absolute path
 		final File file = new File(rootPath, requestPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/css/index.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/css/index.css b/flink-runtime-web/src/main/resources/web/css/index.css
deleted file mode 100644
index 43f7d98..0000000
--- a/flink-runtime-web/src/main/resources/web/css/index.css
+++ /dev/null
@@ -1,596 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#sidebar {
-  overflow: hidden;
-  position: fixed;
-  left: -250px;
-  top: 0;
-  bottom: 0;
-  height: 100%;
-  width: 250px;
-  background: #151515;
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-  -webkit-box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
-  box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
-}
-#sidebar.sidebar-visible {
-  left: 0;
-}
-#sidebar .logo {
-  width: auto;
-  height: 22px;
-}
-#sidebar .logo img {
-  display: inline-block;
-}
-#sidebar .navbar-static-top {
-  overflow: hidden;
-  height: 51px;
-}
-#sidebar .navbar-static-top .navbar-header {
-  width: 100%;
-}
-#sidebar .navbar-brand.navbar-brand-text {
-  font-size: 14px;
-  font-weight: bold;
-  color: #fff;
-  padding-left: 0;
-}
-#sidebar .nav > li > a {
-  color: #aaa;
-  margin-bottom: 1px;
-}
-#sidebar .nav > li > a:hover,
-#sidebar .nav > li > a:focus {
-  background-color: rgba(40,40,40,0.5);
-}
-#sidebar .nav > li > a.active {
-  background-color: rgba(100,100,100,0.5);
-}
-#content {
-  background-color: #fff;
-  overflow: hidden;
-  margin-left: 0;
-  padding-top: 70px;
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-}
-#content .navbar-main,
-#content .navbar-main-additional {
-  -webkit-transition: 400ms;
-  -moz-transition: 400ms;
-  -o-transition: 400ms;
-  -ms-transition: 400ms;
-  transition: 400ms;
-}
-#content .navbar-main-additional {
-  margin-top: 51px;
-  border-bottom: none;
-  padding: 0 20px;
-}
-#content .navbar-main-additional .nav-tabs {
-  margin: 0 -20px;
-  padding: 0 20px;
-}
-#content .navbar-secondary-additional {
-  border: none;
-  padding: 0 20px;
-  margin-bottom: 0;
-}
-#content .navbar-secondary-additional .nav-tabs {
-  margin: 0 -20px;
-}
-#content.sidebar-visible {
-  margin-left: 250px;
-}
-#content.sidebar-visible .navbar-main,
-#content.sidebar-visible .navbar-main-additional {
-  left: 250px;
-}
-#content #fold-button {
-  display: inline-block;
-  margin-left: 20px;
-}
-#content #content-inner {
-  padding: 0px 20px 20px 20px;
-}
-#content #content-inner.has-navbar-main-additional {
-  padding-top: 42px;
-}
-.page-header {
-  margin: 0 0 20px 0;
-}
-.nav > li > a,
-.nav > li > a:hover,
-.nav > li > a:focus {
-  color: #aaa;
-  background-color: transparent;
-  border-bottom: 2px solid transparent;
-}
-.nav > li.active > a,
-.nav > li.active > a:hover,
-.nav > li.active > a:focus {
-  color: #000;
-  border-bottom: 2px solid #000;
-}
-.nav.nav-tabs {
-  margin-bottom: 20px;
-}
-.table .table {
-  background-color: transparent;
-}
-.table th {
-  font-weight: normal;
-  color: #999;
-}
-.table td.td-long {
-  width: 20%;
-  white-space: pre-wrap;
-  white-space: -moz-pre-wrap;
-  white-space: -pre-wrap;
-  white-space: -o-pre-wrap;
-  word-wrap: break-word;
-}
-.table.table-clickable tr {
-  cursor: pointer;
-}
-.table.table-inner {
-  background-color: transparent;
-}
-.table.table-properties {
-  table-layout: fixed;
-  white-space: nowrap;
-}
-.table.table-properties td {
-  width: 50%;
-  white-space: nowrap;
-  overflow: hidden;
-  -o-text-overflow: ellipsis;
-  text-overflow: ellipsis;
-}
-.table.table-body-hover > tbody {
-  border-top: none;
-  border-left: 2px solid transparent;
-}
-.table.table-body-hover > tbody.active {
-  border-left: 2px solid #000;
-}
-.table.table-body-hover > tbody:hover td:not(.tab-column),
-.table.table-body-hover > tbody.active td:not(.tab-column) {
-  background-color: #f0f0f0;
-}
-.table.table-body-hover > tbody:hover td.tab-column li.active,
-.table.table-body-hover > tbody.active td.tab-column li.active {
-  background-color: #f0f0f0;
-}
-.table.table-activable th.tab-column,
-.table.table-activable td.tab-column {
-  border-top: none;
-  width: 47px;
-}
-.table.table-activable td.tab-column {
-  border-right: 1px solid #ddd;
-}
-.table.table-activable td {
-  position: relative;
-}
-.table .small-label {
-  text-transform: uppercase;
-  font-size: 13px;
-  color: #999;
-}
-.panel.panel-dashboard .huge {
-  font-size: 28px;
-}
-.panel.panel-lg {
-  font-size: 16px;
-}
-.panel.panel-lg .badge {
-  font-size: 14px;
-}
-.navbar-secondary {
-  overflow: auto;
-}
-.navbar-main .navbar-title,
-.navbar-secondary .navbar-title,
-.navbar-main-additional .navbar-title,
-.panel.panel-multi .navbar-title,
-.navbar-secondary-additional .navbar-title,
-.navbar-main .panel-title,
-.navbar-secondary .panel-title,
-.navbar-main-additional .panel-title,
-.panel.panel-multi .panel-title,
-.navbar-secondary-additional .panel-title {
-  float: left;
-  font-size: 18px;
-  padding: 12px 20px 13px 10px;
-  color: #333;
-  display: inline-block;
-}
-.navbar-main .navbar-info,
-.navbar-secondary .navbar-info,
-.navbar-main-additional .navbar-info,
-.panel.panel-multi .navbar-info,
-.navbar-secondary-additional .navbar-info,
-.navbar-main .panel-info,
-.navbar-secondary .panel-info,
-.navbar-main-additional .panel-info,
-.panel.panel-multi .panel-info,
-.navbar-secondary-additional .panel-info {
-  float: left;
-  font-size: 14px;
-  padding: 15px 15px 15px 15px;
-  color: #999;
-  display: inline-block;
-  border-right: 1px solid #e7e7e7;
-  overflow: hidden;
-}
-.navbar-main .navbar-info .overflow,
-.navbar-secondary .navbar-info .overflow,
-.navbar-main-additional .navbar-info .overflow,
-.panel.panel-multi .navbar-info .overflow,
-.navbar-secondary-additional .navbar-info .overflow,
-.navbar-main .panel-info .overflow,
-.navbar-secondary .panel-info .overflow,
-.navbar-main-additional .panel-info .overflow,
-.panel.panel-multi .panel-info .overflow,
-.navbar-secondary-additional .panel-info .overflow {
-  position: absolute;
-  display: block;
-  -o-text-overflow: ellipsis;
-  text-overflow: ellipsis;
-  overflow: hidden;
-  height: 22px;
-  line-height: 22px;
-  vertical-align: middle;
-}
-.navbar-main .navbar-info.first,
-.navbar-secondary .navbar-info.first,
-.navbar-main-additional .navbar-info.first,
-.panel.panel-multi .navbar-info.first,
-.navbar-secondary-additional .navbar-info.first,
-.navbar-main .panel-info.first,
-.navbar-secondary .panel-info.first,
-.navbar-main-additional .panel-info.first,
-.panel.panel-multi .panel-info.first,
-.navbar-secondary-additional .panel-info.first {
-  border-left: 1px solid #e7e7e7;
-}
-.navbar-main .navbar-info.last,
-.navbar-secondary .navbar-info.last,
-.navbar-main-additional .navbar-info.last,
-.panel.panel-multi .navbar-info.last,
-.navbar-secondary-additional .navbar-info.last,
-.navbar-main .panel-info.last,
-.navbar-secondary .panel-info.last,
-.navbar-main-additional .panel-info.last,
-.panel.panel-multi .panel-info.last,
-.navbar-secondary-additional .panel-info.last {
-  border-right: none;
-}
-.panel.panel-multi .panel-heading {
-  padding: 0;
-}
-.panel.panel-multi .panel-heading .panel-info.thin {
-  padding: 8px 10px;
-}
-.panel.panel-multi .panel-body {
-  padding: 10px;
-  background-color: #fdfdfd;
-  color: #999;
-  font-size: 13px;
-}
-.panel.panel-multi .panel-body.clean {
-  color: inherit;
-  font-size: inherit;
-}
-.navbar-main-additional,
-.navbar-secondary-additional {
-  min-height: 40px;
-  background-color: #fdfdfd;
-}
-.navbar-main-additional .navbar-info,
-.navbar-secondary-additional .navbar-info {
-  font-size: 13px;
-  padding: 10px 15px 10px 15px;
-}
-.nav-top-affix.affix {
-  width: 100%;
-  top: 50px;
-  margin-left: -20px;
-  padding-left: 20px;
-  margin-right: -20px;
-  padding-right: 20px;
-  background-color: #fff;
-  z-index: 1;
-}
-.badge-default[href]:hover,
-.badge-default[href]:focus {
-  background-color: #808080;
-}
-.badge-primary {
-  background-color: #428bca;
-}
-.badge-primary[href]:hover,
-.badge-primary[href]:focus {
-  background-color: #3071a9;
-}
-.badge-success {
-  background-color: #5cb85c;
-}
-.badge-success[href]:hover,
-.badge-success[href]:focus {
-  background-color: #449d44;
-}
-.badge-info {
-  background-color: #5bc0de;
-}
-.badge-info[href]:hover,
-.badge-info[href]:focus {
-  background-color: #31b0d5;
-}
-.badge-warning {
-  background-color: #f0ad4e;
-}
-.badge-warning[href]:hover,
-.badge-warning[href]:focus {
-  background-color: #ec971f;
-}
-.badge-danger {
-  background-color: #d9534f;
-}
-.badge-danger[href]:hover,
-.badge-danger[href]:focus {
-  background-color: #c9302c;
-}
-.indicator {
-  display: inline-block;
-  margin-right: 15px;
-}
-.indicator.indicator-primary {
-  color: #428bca;
-}
-.indicator.indicator-success {
-  color: #5cb85c;
-}
-.indicator.indicator-info {
-  color: #5bc0de;
-}
-.indicator.indicator-warning {
-  color: #f0ad4e;
-}
-.indicator.indicator-danger {
-  color: #d9534f;
-}
-pre.exception {
-  border: none;
-  background-color: transparent;
-  padding: 0;
-  margin: 0;
-}
-.nav-tabs.tabs-vertical {
-  position: absolute;
-  left: 0;
-  top: 0;
-  border-bottom: none;
-  z-index: 100;
-}
-.nav-tabs.tabs-vertical li {
-  float: none;
-  margin-bottom: 0;
-  margin-right: -1px;
-}
-.nav-tabs.tabs-vertical li > a {
-  margin-right: 0;
-  -webkit-border-radius: 0;
-  border-radius: 0;
-  border-bottom: none;
-  border-left: 2px solid transparent;
-}
-.nav-tabs.tabs-vertical li > a:hover,
-.nav-tabs.tabs-vertical li > a:focus {
-  border-bottom: none;
-  border-left: 2px solid #000;
-}
-.nav-tabs.tabs-vertical li.active > a {
-  border-bottom: none;
-  border-left: 2px solid #000;
-}
-.navbar-main .navbar-title,
-.navbar-secondary .navbar-title,
-.navbar-main-additional .navbar-title,
-.navbar-secondary-additional .navbar-title {
-  padding: 12px 20px 13px 20px;
-}
-livechart {
-  width: 30%;
-  height: 30%;
-  text-align: center;
-}
-.canvas-wrapper {
-  border: 1px solid #ddd;
-  position: relative;
-  margin-bottom: 20px;
-}
-.canvas-wrapper .main-canvas {
-  height: 400px;
-  overflow: hidden;
-}
-.canvas-wrapper .main-canvas .zoom-buttons {
-  position: absolute;
-  top: 10px;
-  right: 10px;
-}
-.label-group .label {
-  display: inline-block;
-  width: 2em;
-  padding-left: 0.1em;
-  padding-right: 0.1em;
-  margin: 0;
-  border-right: 1px solid #fff;
-  -webkit-border-radius: 0;
-  border-radius: 0;
-}
-.label-group .label.label-black {
-  background-color: #000;
-}
-svg.graph {
-  overflow: hidden;
-}
-svg.graph g.type-TK > rect {
-  fill: #00ffd0;
-}
-svg.graph text {
-  font-weight: 300;
-  font-size: 14px;
-}
-svg.graph .node {
-  cursor: pointer;
-}
-svg.graph .node > rect {
-  stroke: #999;
-  stroke-width: 5px;
-  fill: #fff;
-  margin: 0;
-  padding: 0;
-}
-svg.graph .node[active] > rect {
-  fill: #eee;
-}
-svg.graph .node.node-mirror > rect {
-  stroke: #a8a8a8;
-}
-svg.graph .node.node-iteration > rect {
-  stroke: #cd3333;
-}
-svg.graph .node.node-source > rect {
-  stroke: #4ce199;
-}
-svg.graph .node.node-sink > rect {
-  stroke: #e6ec8b;
-}
-svg.graph .node.node-normal > rect {
-  stroke: #3fb6d8;
-}
-svg.graph .node h4 {
-  color: #000;
-}
-svg.graph .node h5 {
-  color: #999;
-}
-svg.graph .edgeLabel rect {
-  fill: #fff;
-}
-svg.graph .edgePath path {
-  stroke: #333;
-  stroke-width: 2px;
-  fill: #333;
-}
-svg.graph .label {
-  color: #777;
-  margin: 0;
-}
-svg.graph .edge-label {
-  font-size: 14px;
-}
-svg.graph .node-label {
-  display: block;
-  margin: 0;
-  text-decoration: none;
-}
-.timeline {
-  overflow: hidden;
-}
-.timeline-canvas {
-  overflow: hidden;
-  padding: 10px;
-}
-.timeline-canvas .bar-container {
-  overflow: hidden;
-}
-.timeline-canvas .timeline-insidelabel,
-.timeline-canvas .timeline-series {
-  cursor: pointer;
-}
-.timeline-canvas.secondary .timeline-insidelabel,
-.timeline-canvas.secondary .timeline-series {
-  cursor: auto;
-}
-.qtip-timeline-bar {
-  font-size: 14px;
-  line-height: 1.4;
-}
-@media (min-width: 1024px) and (max-width: 1279px) {
-  #sidebar {
-    left: 0;
-    width: 160px;
-  }
-  #sidebar .navbar-static-top .navbar-brand-text {
-    display: none;
-  }
-  #content {
-    margin-left: 160px;
-  }
-  #content #fold-button {
-    display: none;
-  }
-  #content .navbar-main,
-  #content .navbar-main-additional {
-    left: 160px;
-  }
-  .table td.td-long {
-    width: 20%;
-  }
-}
-@media (min-width: 1280px) {
-  #sidebar {
-    left: 0;
-  }
-  #content {
-    margin-left: 250px;
-  }
-  #content #fold-button {
-    display: none;
-  }
-  #content .navbar-main,
-  #content .navbar-main-additional {
-    left: 250px;
-  }
-  .table td.td-long {
-    width: 30%;
-  }
-}
-#total-mem {
-  background-color: #7cb5ec;
-}
-#heap-mem {
-  background-color: #434348;
-}
-#non-heap-mem {
-  background-color: #90ed7d;
-}
-a.show-pointer {
-  cursor: pointer;
-}


[03/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
index 837b643..cd40c82 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
@@ -26,10 +26,10 @@ import org.apache.flink.runtime.io.network.api.reader.RecordReader;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
-import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CurrentJobStatus;
 import org.apache.flink.runtime.messages.JobManagerMessages.JobNotFound;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.types.IntValue;
 import org.junit.Test;
 import scala.concurrent.Await;
@@ -48,7 +49,6 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import static org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
 import static org.apache.flink.runtime.messages.JobManagerMessages.RequestJobStatus;
-import static org.junit.Assert.fail;
 
 public class TaskCancelTest {
 
@@ -109,30 +109,21 @@ public class TaskCancelTest {
 
 			// Wait for the job to make some progress and then cancel
 			awaitRunning(
-				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
-				jobGraph.getJobID(),
-				TestingUtils.TESTING_DURATION());
+					flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+					jobGraph.getJobID(),
+					TestingUtils.TESTING_DURATION());
 
 			Thread.sleep(5000);
 
 			cancelJob(
-				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
-				jobGraph.getJobID(),
-				TestingUtils.TESTING_DURATION());
+					flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+					jobGraph.getJobID(),
+					TestingUtils.TESTING_DURATION());
 
 			// Wait for the job to be cancelled
-			JobStatus status = awaitTermination(
-				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
-				jobGraph.getJobID(),
-				TestingUtils.TESTING_DURATION());
-
-			if (status == JobStatus.CANCELED) {
-				// Expected :-) All is swell.
-			}
-			else {
-				fail("The job finished with unexpected terminal state " + status + ". "
-						+ "This indicates that there is a bug in the task cancellation.");
-			}
+			JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.CANCELED,
+					flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+					TestingUtils.TESTING_DURATION());
 		}
 		finally {
 			if (flink != null) {
@@ -224,42 +215,6 @@ public class TaskCancelTest {
 
 	}
 
-	private JobStatus awaitTermination(ActorGateway jobManager, JobID jobId, FiniteDuration timeout)
-			throws Exception {
-
-		checkNotNull(jobManager);
-		checkNotNull(jobId);
-		checkNotNull(timeout);
-
-		while (true) {
-			Future<Object> ask = jobManager.ask(
-					new RequestJobStatus(jobId),
-					timeout);
-
-			Object result = Await.result(ask, timeout);
-
-			if (result instanceof CurrentJobStatus) {
-				// Success
-				CurrentJobStatus status = (CurrentJobStatus) result;
-
-				if (!status.jobID().equals(jobId)) {
-					throw new Exception("JobManager responded for wrong job ID. Request: "
-							+ jobId + ", response: " + status.jobID() + ".");
-				}
-
-				if (status.status().isTerminalState()) {
-					return status.status();
-				}
-			}
-			else if (result instanceof JobNotFound) {
-				throw new Exception("Cannot find job with ID " + jobId + ".");
-			}
-			else {
-				throw new Exception("Unexpected response to cancel request: " + result);
-			}
-		}
-	}
-
 	// ---------------------------------------------------------------------------------------------
 
 	public static class InfiniteSource extends AbstractInvokable {

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
index 61b1f7a..069b6af 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
@@ -18,26 +18,17 @@
 
 package org.apache.flink.runtime.testutils;
 
-import static org.junit.Assert.fail;
+import org.apache.flink.runtime.util.FileUtils;
 
-import java.io.BufferedWriter;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
+import java.io.InputStream;
 import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import java.util.UUID;
 
 /**
  * This class contains auxiliary methods for unit tests.
@@ -77,6 +68,18 @@ public class CommonTestUtils {
 	}
 
 	/**
+	 * Create a temporary log4j configuration for the test.
+	 */
+	public static File createTemporaryLog4JProperties() throws IOException {
+		File log4jProps = File.createTempFile(FileUtils.getRandomFilename(""), "-log4j" +
+				".properties");
+		log4jProps.deleteOnExit();
+		CommonTestUtils.printLog4jDebugConfig(log4jProps);
+
+		return log4jProps;
+	}
+
+	/**
 	 * Tries to get the java executable command with which the current JVM was started.
 	 * Returns null, if the command could not be found.
 	 *
@@ -152,4 +155,50 @@ public class CommonTestUtils {
 			fw.close();
 		}
 	}
+
+	public static File createTempDirectory() throws IOException {
+		File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+		for (int i = 0; i < 10; i++) {
+			File dir = new File(tempDir, UUID.randomUUID().toString());
+			if (!dir.exists() && dir.mkdirs()) {
+				return dir;
+			}
+			System.err.println("Could not use temporary directory " + dir.getAbsolutePath());
+		}
+
+		throw new IOException("Could not create temporary file directory");
+	}
+
+	/**
+	 * Utility class to read the output of a process stream and forward it into a StringWriter.
+	 */
+	public static class PipeForwarder extends Thread {
+
+		private final StringWriter target;
+		private final InputStream source;
+
+		public PipeForwarder(InputStream source, StringWriter target) {
+			super("Pipe Forwarder");
+			setDaemon(true);
+
+			this.source = source;
+			this.target = target;
+
+			start();
+		}
+
+		@Override
+		public void run() {
+			try {
+				int next;
+				while ((next = source.read()) != -1) {
+					target.write(next);
+				}
+			}
+			catch (IOException e) {
+				// terminate
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java
new file mode 100644
index 0000000..66e1d9b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testutils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.messages.JobManagerMessages.CurrentJobStatus;
+import org.apache.flink.runtime.messages.JobManagerMessages.JobStatusResponse;
+import org.apache.flink.runtime.testingUtils.TestingJobManager;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.flink.runtime.messages.JobManagerMessages.JobNotFound;
+import static org.apache.flink.runtime.messages.JobManagerMessages.RequestJobStatus;
+import static org.apache.flink.runtime.messages.JobManagerMessages.getRequestJobStatus;
+import static org.apache.flink.runtime.messages.JobManagerMessages.getRequestNumberRegisteredTaskManager;
+
+/**
+ * JobManager actor test utilities.
+ *
+ * <p>If you are using a {@link TestingJobManager} most of these are not needed.
+ */
+public class JobManagerActorTestUtils {
+
+	/**
+	 * Waits for the expected {@link JobStatus}.
+	 *
+	 * <p>Repeatedly queries the JobManager via {@link RequestJobStatus} messages.
+	 *
+	 * @param jobId             Job ID of the job to wait for
+	 * @param expectedJobStatus Expected job status
+	 * @param jobManager        Job manager actor to ask
+	 * @param timeout           Timeout after which the operation fails
+	 * @throws Exception If the job is not found within the timeout or the job is in another state.
+	 */
+	public static void waitForJobStatus(
+			JobID jobId,
+			JobStatus expectedJobStatus,
+			ActorGateway jobManager,
+			FiniteDuration timeout) throws Exception {
+
+		checkNotNull(jobId, "Job ID");
+		checkNotNull(expectedJobStatus, "Expected job status");
+		checkNotNull(jobManager, "Job manager");
+		checkNotNull(timeout, "Timeout");
+
+		final Deadline deadline = timeout.fromNow();
+
+		while (deadline.hasTimeLeft()) {
+			// Request the job status
+			JobStatusResponse response = requestJobStatus(jobId, jobManager, deadline.timeLeft());
+
+			// Found the job
+			if (response instanceof CurrentJobStatus) {
+				JobStatus jobStatus = ((CurrentJobStatus) response).status();
+
+				// OK, that's what we were waiting for
+				if (jobStatus == expectedJobStatus) {
+					return;
+				}
+				else if (jobStatus.isTerminalState()) {
+					throw new IllegalStateException("Job is in terminal state " + jobStatus + ", "
+							+ "but was waiting for " + expectedJobStatus + ".");
+				}
+			}
+			// Did not find the job... retry
+			else if (response instanceof JobNotFound) {
+				Thread.sleep(Math.min(100, deadline.timeLeft().toMillis()));
+			}
+			else {
+				throw new IllegalStateException("Unexpected response.");
+			}
+		}
+
+		throw new IllegalStateException("Job not found within deadline.");
+	}
+
+	/**
+	 * Request a {@link JobStatusResponse}.
+	 *
+	 * @param jobId      Job ID of the job to request the status of
+	 * @param jobManager Job manager actor to ask
+	 * @param timeout    Timeout after which the operation fails
+	 * @return The {@link JobStatusResponse} from the job manager
+	 * @throws Exception If there is no answer within the timeout.
+	 */
+	public static JobStatusResponse requestJobStatus(
+			JobID jobId,
+			ActorGateway jobManager,
+			FiniteDuration timeout) throws Exception {
+
+		checkNotNull(jobId, "Job ID");
+		checkNotNull(jobManager, "Job manager");
+		checkNotNull(timeout, "Timeout");
+
+		// Ask the JobManager
+		RequestJobStatus request = (RequestJobStatus) getRequestJobStatus(jobId);
+		Future<Object> ask = jobManager.ask(request, timeout);
+		Object response = Await.result(ask, timeout);
+
+		if (response instanceof JobStatusResponse) {
+			return (JobStatusResponse) response;
+		}
+
+		throw new IllegalStateException("Unexpected response.");
+	}
+
+	/**
+	 * Waits for a minimum number of task managers to connect to the job manager.
+	 *
+	 * @param minimumNumberOfTaskManagers Minimum number of task managers to wait for
+	 * @param jobManager                  Job manager actor to ask
+	 * @param timeout                     Timeout after which the operation fails
+	 * @throws Exception If the task managers don't connection with the timeout.
+	 */
+	public static void waitForTaskManagers(
+			int minimumNumberOfTaskManagers,
+			ActorGateway jobManager,
+			FiniteDuration timeout) throws Exception {
+
+		checkArgument(minimumNumberOfTaskManagers >= 1);
+		checkNotNull(jobManager, "Job manager");
+		checkNotNull(timeout, "Timeout");
+
+		final Deadline deadline = timeout.fromNow();
+
+		while (deadline.hasTimeLeft()) {
+			Future<Object> ask = jobManager.ask(getRequestNumberRegisteredTaskManager(),
+					deadline.timeLeft());
+
+			Integer response = (Integer) Await.result(ask, deadline.timeLeft());
+
+			// All are connected. We are done.
+			if (response >= minimumNumberOfTaskManagers) {
+				return;
+			}
+			// Waiting for more... retry
+			else {
+				Thread.sleep(Math.min(100, deadline.timeLeft().toMillis()));
+			}
+		}
+
+		throw new IllegalStateException("Task managers not connected within deadline.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
new file mode 100644
index 0000000..85b768d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
@@ -0,0 +1,226 @@
+/*
+ * 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.testutils;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmanager.JobManagerMode;
+import org.apache.flink.util.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A {@link JobManager} instance running in a separate JVM.
+ */
+public class JobManagerProcess extends TestJvmProcess {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JobManagerProcess.class);
+
+	/** ID for this JobManager */
+	private final int id;
+
+	/** The port the JobManager listens on */
+	private final int jobManagerPort;
+
+	/** The configuration for the JobManager */
+	private final Configuration config;
+
+	/** Configuration parsed as args for {@link JobManagerProcess.JobManagerProcessEntryPoint} */
+	private final String[] jvmArgs;
+
+	private ActorRef jobManagerRef;
+
+	/**
+	 * Creates a {@link JobManager} running in a separate JVM.
+	 *
+	 * <p>See {@link #JobManagerProcess(int, Configuration, int)} for a more
+	 * detailed
+	 * description.
+	 *
+	 * @param config Configuration for the job manager process
+	 * @throws Exception
+	 */
+	public JobManagerProcess(int id, Configuration config) throws Exception {
+		this(id, config, 0);
+	}
+
+	/**
+	 * Creates a {@link JobManager} running in a separate JVM.
+	 *
+	 * @param id             ID for the JobManager
+	 * @param config         Configuration for the job manager process
+	 * @param jobManagerPort Job manager port (if <code>0</code>, pick any available port)
+	 * @throws Exception
+	 */
+	public JobManagerProcess(int id, Configuration config, int jobManagerPort) throws Exception {
+		checkArgument(id >= 0, "Negative ID");
+		this.id = id;
+		this.config = checkNotNull(config, "Configuration");
+		this.jobManagerPort = jobManagerPort <= 0 ? NetUtils.getAvailablePort() : jobManagerPort;
+
+		ArrayList<String> args = new ArrayList<>();
+		args.add("--port");
+		args.add(String.valueOf(this.jobManagerPort));
+
+		for (Map.Entry<String, String> entry : config.toMap().entrySet()) {
+			args.add("--" + entry.getKey());
+			args.add(entry.getValue());
+		}
+
+		this.jvmArgs = new String[args.size()];
+		args.toArray(jvmArgs);
+	}
+
+	@Override
+	public String getName() {
+		return "JobManager " + id;
+	}
+
+	@Override
+	public String[] getJvmArgs() {
+		return jvmArgs;
+	}
+
+	@Override
+	public String getEntryPointClassName() {
+		return JobManagerProcessEntryPoint.class.getName();
+	}
+
+	public int getJobManagerPort() {
+		return jobManagerPort;
+	}
+
+	public Configuration getConfig() {
+		return config;
+	}
+
+	/**
+	 * Returns the Akka URL of this JobManager.
+	 */
+	public String getJobManagerAkkaURL() {
+		return JobManager.getRemoteJobManagerAkkaURL(
+				new InetSocketAddress("localhost", jobManagerPort),
+				Option.<String>empty());
+	}
+
+	@Override
+	public String toString() {
+		return String.format("JobManagerProcess(id=%d, port=%d)", id, jobManagerPort);
+	}
+
+	/**
+	 * Waits for the job manager to be reachable.
+	 *
+	 * <p><strong>Important:</strong> Make sure to set the timeout larger than Akka's gating
+	 * time. Otherwise, this will effectively not wait for the JobManager to startup, because the
+	 * time out will fire immediately.
+	 *
+	 * @param actorSystem Actor system to be used to resolve JobManager address.
+	 * @param timeout     Timeout (make sure to set larger than Akka's gating time).
+	 */
+	public ActorRef getActorRef(ActorSystem actorSystem, FiniteDuration timeout)
+			throws Exception {
+
+		if (jobManagerRef != null) {
+			return jobManagerRef;
+		}
+
+		checkNotNull(actorSystem, "Actor system");
+
+		// Deadline passes timeout ms
+		Deadline deadline = timeout.fromNow();
+
+		while (deadline.hasTimeLeft()) {
+			try {
+				// If the Actor is not reachable yet, this throws an Exception. Retry until the
+				// deadline passes.
+				this.jobManagerRef = AkkaUtils.getActorRef(
+						getJobManagerAkkaURL(),
+						actorSystem,
+						deadline.timeLeft());
+
+				return jobManagerRef;
+			}
+			catch (Throwable ignored) {
+				// Retry
+				Thread.sleep(Math.min(100, deadline.timeLeft().toMillis()));
+			}
+		}
+
+		throw new IllegalStateException("JobManager did not start up within " + timeout + ".");
+	}
+
+	/**
+	 * Entry point for the JobManager process.
+	 */
+	public static class JobManagerProcessEntryPoint {
+
+		private static final Logger LOG = LoggerFactory.getLogger(JobManagerProcessEntryPoint.class);
+
+		/**
+		 * Runs the JobManager process in {@link JobManagerMode#CLUSTER} and {@link
+		 * StreamingMode#STREAMING} (can handle both batch and streaming jobs).
+		 *
+		 * <p><strong>Required argument</strong>: <code>port</code>. Start the process with
+		 * <code>--port PORT</code>.
+		 *
+		 * <p>Other arguments are parsed to a {@link Configuration} and passed to the
+		 * JobManager, for instance: <code>--recovery.mode ZOOKEEPER --ha.zookeeper.quorum
+		 * "xyz:123:456"</code>.
+		 */
+		public static void main(String[] args) {
+			try {
+				ParameterTool params = ParameterTool.fromArgs(args);
+				final int port = Integer.valueOf(params.getRequired("port"));
+				LOG.info("Running on port {}.", port);
+
+				Configuration config = params.getConfiguration();
+				LOG.info("Configuration: {}.", config);
+
+				// Run the JobManager
+				JobManager.runJobManager(config, JobManagerMode.CLUSTER, StreamingMode.STREAMING,
+						"localhost", port);
+
+				// Run forever. Forever, ever? Forever, ever!
+				new CountDownLatch(1).await();
+			}
+			catch (Throwable t) {
+				LOG.error("Failed to start JobManager process", t);
+				System.exit(1);
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java
new file mode 100644
index 0000000..f683c55
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testutils;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A {@link TaskManager} instance running in a separate JVM.
+ */
+public class TaskManagerProcess extends TestJvmProcess {
+
+	/** ID for this TaskManager */
+	private final int id;
+
+	/** The configuration for the TaskManager */
+	private final Configuration config;
+
+	/** Configuration parsed as args for {@link TaskManagerProcess.TaskManagerProcessEntryPoint} */
+	private final String[] jvmArgs;
+
+	public TaskManagerProcess(int id, Configuration config) throws Exception {
+		checkArgument(id >= 0, "Negative ID");
+		this.id = id;
+		this.config = checkNotNull(config, "Configuration");
+
+		ArrayList<String> args = new ArrayList<>();
+
+		for (Map.Entry<String, String> entry : config.toMap().entrySet()) {
+			args.add("--" + entry.getKey());
+			args.add(entry.getValue());
+		}
+
+		this.jvmArgs = new String[args.size()];
+		args.toArray(jvmArgs);
+	}
+
+	@Override
+	public String getName() {
+		return "TaskManager " + id;
+	}
+
+	@Override
+	public String[] getJvmArgs() {
+		return jvmArgs;
+	}
+
+	@Override
+	public String getEntryPointClassName() {
+		return TaskManagerProcessEntryPoint.class.getName();
+	}
+
+	public int getId() {
+		return id;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("TaskManagerProcess(id=%d)", id);
+	}
+
+	/**
+	 * Entry point for the TaskManager process.
+	 */
+	public static class TaskManagerProcessEntryPoint {
+
+		private static final Logger LOG = LoggerFactory.getLogger(TaskManagerProcessEntryPoint.class);
+
+		/**
+		 * Runs the JobManager process in {@link StreamingMode#STREAMING} (can handle both batch
+		 * and streaming jobs).
+		 *
+		 * <p>All arguments are parsed to a {@link Configuration} and passed to the Taskmanager,
+		 * for instance: <code>--recovery.mode ZOOKEEPER --ha.zookeeper.quorum "xyz:123:456"</code>.
+		 */
+		public static void main(String[] args) throws Exception {
+			try {
+				Configuration config = ParameterTool.fromArgs(args).getConfiguration();
+
+				if (!config.containsKey(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY)) {
+					config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
+				}
+
+				if (!config.containsKey(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY)) {
+					config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
+				}
+
+
+				LOG.info("Configuration: {}.", config);
+
+				// Run the TaskManager
+				TaskManager.selectNetworkInterfaceAndRunTaskManager(
+						config, StreamingMode.STREAMING, TaskManager.class);
+
+				// Run forever
+				new CountDownLatch(1).await();
+			}
+			catch (Throwable t) {
+				LOG.error("Failed to start TaskManager process", t);
+				System.exit(1);
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java
new file mode 100644
index 0000000..0920b5c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java
@@ -0,0 +1,267 @@
+/*
+ * 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.testutils;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Arrays;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.fail;
+
+/**
+ * A {@link Process} running a separate JVM.
+ */
+public abstract class TestJvmProcess {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TestJvmProcess.class);
+
+	/** Lock to guard {@link #createAndStart()} and {@link #destroy()} calls. */
+	private final Object createDestroyLock = new Object();
+
+	/** The java command path */
+	private final String javaCommandPath;
+
+	/** The log4j configuration path. */
+	private final String log4jConfigFilePath;
+
+	/** Shutdown hook for resource cleanup */
+	private final Thread shutdownHook;
+
+	/** JVM process memory (set for both '-Xms' and '-Xmx'). */
+	private int jvmMemoryInMb = 80;
+
+	/** The JVM process */
+	private Process process;
+
+	/** Writer for the process output */
+	private volatile StringWriter processOutput;
+
+	public TestJvmProcess() throws Exception {
+		this(getJavaCommandPath(), createTemporaryLog4JProperties().getPath());
+	}
+
+	public TestJvmProcess(String javaCommandPath, String log4jConfigFilePath) {
+		this.javaCommandPath = checkNotNull(javaCommandPath, "Java command path");
+		this.log4jConfigFilePath = checkNotNull(log4jConfigFilePath, "log4j config file path");
+
+		this.shutdownHook = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					destroy();
+				}
+				catch (Throwable t) {
+					LOG.error("Error during process cleanup shutdown hook.", t);
+				}
+			}
+		});
+	}
+
+	/**
+	 * Returns the name of the process.
+	 */
+	public abstract String getName();
+
+	/**
+	 * Returns the arguments to the JVM.
+	 *
+	 * <p>These can be parsed by the main method of the entry point class.
+	 */
+	public abstract String[] getJvmArgs();
+
+	/**
+	 * Returns the name of the class to run.
+	 *
+	 * <p>Arguments to the main method can be specified via {@link #getJvmArgs()}.
+	 */
+	public abstract String getEntryPointClassName();
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the memory for the process (<code>-Xms</code> and <code>-Xmx</code> flags) (>= 80).
+	 *
+	 * @param jvmMemoryInMb Amount of memory in Megabytes for the JVM (>= 80).
+	 */
+	public void setJVMMemory(int jvmMemoryInMb) {
+		checkArgument(jvmMemoryInMb >= 80, "JobManager JVM Requires at least 80 MBs of memory.");
+		this.jvmMemoryInMb = jvmMemoryInMb;
+	}
+
+	/**
+	 * Creates and starts the {@link Process}.
+	 *
+	 * <strong>Important:</strong> Don't forget to call {@link #destroy()} to prevent
+	 * resource leaks. The created process will be child process and is not guaranteed to
+	 * terminate when the parent process terminates.
+	 */
+	public void createAndStart() throws IOException {
+		String[] cmd = new String[] {
+				javaCommandPath,
+				"-Dlog.level=DEBUG",
+				"-Dlog4j.configuration=file:" + log4jConfigFilePath,
+				"-Xms" + jvmMemoryInMb + "m",
+				"-Xmx" + jvmMemoryInMb + "m",
+				"-classpath", getCurrentClasspath(),
+				getEntryPointClassName() };
+
+		String[] jvmArgs = getJvmArgs();
+
+		if (jvmArgs != null && jvmArgs.length > 0) {
+			cmd = ArrayUtils.addAll(cmd, jvmArgs);
+		}
+
+		synchronized (createDestroyLock) {
+			if (process == null) {
+				LOG.debug("Running command '{}'.", Arrays.toString(cmd));
+				this.process = new ProcessBuilder(cmd).start();
+
+				// Forward output
+				this.processOutput = new StringWriter();
+				new CommonTestUtils.PipeForwarder(process.getErrorStream(), processOutput);
+
+				try {
+					// Add JVM shutdown hook to call shutdown of service
+					Runtime.getRuntime().addShutdownHook(shutdownHook);
+				}
+				catch (IllegalStateException ignored) {
+					// JVM is already shutting down. No need to do this.
+				}
+				catch (Throwable t) {
+					LOG.error("Cannot register process cleanup shutdown hook.", t);
+				}
+			}
+			else {
+				throw new IllegalStateException("Already running.");
+			}
+		}
+	}
+
+	public void printProcessLog() {
+		if (processOutput == null) {
+			throw new IllegalStateException("Not started");
+		}
+
+		System.out.println("-----------------------------------------");
+		System.out.println(" BEGIN SPAWNED PROCESS LOG FOR " + getName());
+		System.out.println("-----------------------------------------");
+
+		String out = processOutput.toString();
+		if (out == null || out.length() == 0) {
+			System.out.println("(EMPTY)");
+		}
+		else {
+			System.out.println(out);
+		}
+
+		System.out.println("-----------------------------------------");
+		System.out.println("		END SPAWNED PROCESS LOG " + getName());
+		System.out.println("-----------------------------------------");
+	}
+
+	public void destroy() {
+		synchronized (createDestroyLock) {
+			if (process != null) {
+				LOG.debug("Destroying " + getName() + " process.");
+
+				try {
+					process.destroy();
+				}
+				catch (Throwable t) {
+					LOG.error("Error while trying to destroy process.", t);
+				}
+				finally {
+					process = null;
+
+					if (shutdownHook != null && shutdownHook != Thread.currentThread()) {
+						try {
+							Runtime.getRuntime().removeShutdownHook(shutdownHook);
+						}
+						catch (IllegalStateException ignored) {
+							// JVM is in shutdown already, we can safely ignore this.
+						}
+						catch (Throwable t) {
+							LOG.warn("Exception while unregistering prcess cleanup shutdown hook.");
+						}
+					}
+				}
+			}
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// File based synchronization utilities
+	// ---------------------------------------------------------------------------------------------
+
+	public static void touchFile(File file) throws IOException {
+		if (!file.exists()) {
+			new FileOutputStream(file).close();
+		}
+		if (!file.setLastModified(System.currentTimeMillis())) {
+			throw new IOException("Could not touch the file.");
+		}
+	}
+
+	public static void waitForMarkerFiles(File basedir, String prefix, int num, long timeout) {
+		long now = System.currentTimeMillis();
+		final long deadline = now + timeout;
+
+
+		while (now < deadline) {
+			boolean allFound = true;
+
+			for (int i = 0; i < num; i++) {
+				File nextToCheck = new File(basedir, prefix + i);
+				if (!nextToCheck.exists()) {
+					allFound = false;
+					break;
+				}
+			}
+
+			if (allFound) {
+				return;
+			}
+			else {
+				// not all found, wait for a bit
+				try {
+					Thread.sleep(10);
+				}
+				catch (InterruptedException e) {
+					throw new RuntimeException(e);
+				}
+
+				now = System.currentTimeMillis();
+			}
+		}
+
+		fail("The tasks were not started within time (" + timeout + "msecs)");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
new file mode 100644
index 0000000..d2e5b6a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
@@ -0,0 +1,94 @@
+/*
+ * 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.testutils;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * ZooKeeper test utilities.
+ */
+public class ZooKeeperTestUtils {
+
+	/**
+	 * Creates a configuration to operate in {@link RecoveryMode#ZOOKEEPER}.
+	 *
+	 * @param zooKeeperQuorum   ZooKeeper quorum to connect to
+	 * @param fsStateHandlePath Base path for file system state backend (for checkpoints and
+	 *                          recovery)
+	 * @return A new configuration to operate in {@link RecoveryMode#ZOOKEEPER}.
+	 */
+	public static Configuration createZooKeeperRecoveryModeConfig(
+			String zooKeeperQuorum, String fsStateHandlePath) {
+
+		return setZooKeeperRecoveryMode(new Configuration(), zooKeeperQuorum, fsStateHandlePath);
+	}
+
+	/**
+	 * Sets all necessary configuration keys to operate in {@link RecoveryMode#ZOOKEEPER}.
+	 *
+	 * @param config            Configuration to use
+	 * @param zooKeeperQuorum   ZooKeeper quorum to connect to
+	 * @param fsStateHandlePath Base path for file system state backend (for checkpoints and
+	 *                          recovery)
+	 * @return The modified configuration to operate in {@link RecoveryMode#ZOOKEEPER}.
+	 */
+	public static Configuration setZooKeeperRecoveryMode(
+			Configuration config,
+			String zooKeeperQuorum,
+			String fsStateHandlePath) {
+
+		checkNotNull(config, "Configuration");
+		checkNotNull(zooKeeperQuorum, "ZooKeeper quorum");
+		checkNotNull(fsStateHandlePath, "File state handle backend path");
+
+		// Web frontend, you have been dismissed. Sorry.
+		config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1);
+
+		// ZooKeeper recovery mode
+		config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+		config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeperQuorum);
+
+		int connTimeout = 5000;
+		if (System.getenv().get("CI") != null) {
+			// The regular timeout is to aggressive for Travis and connections are often lost.
+			connTimeout = 20000;
+		}
+
+		config.setInteger(ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout);
+		config.setInteger(ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
+
+		// File system state backend
+		config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
+		config.setString(ConfigConstants.STATE_BACKEND_FS_DIR, fsStateHandlePath + "/checkpoints");
+		config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, fsStateHandlePath + "/recovery");
+
+		// Akka failure detection and execution retries
+		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms");
+		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s");
+		config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
+		config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "10 s");
+
+		return config;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
new file mode 100644
index 0000000..f0130ec
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java
@@ -0,0 +1,591 @@
+/*
+ * 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.zookeeper;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TestLogger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.Stat;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for basic {@link ZooKeeperStateHandleStore} behaviour.
+ *
+ * <p> Tests include:
+ * <ul>
+ * <li>Expected usage of operations</li>
+ * <li>Correct ordering of ZooKeeper and state handle operations</li>
+ * </ul>
+ */
+public class ZooKeeperStateHandleStoreITCase extends TestLogger {
+
+	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (ZooKeeper != null) {
+			ZooKeeper.shutdown();
+		}
+	}
+
+	@Before
+	public void cleanUp() throws Exception {
+		ZooKeeper.deleteAll();
+	}
+
+	/**
+	 * Tests add operation with default {@link CreateMode}.
+	 */
+	@Test
+	public void testAdd() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testAdd";
+		final Long state = 1239712317L;
+
+		// Test
+		store.add(pathInZooKeeper, state);
+
+		// Verify
+		// State handle created
+		assertEquals(1, stateHandleProvider.getStateHandles().size());
+		assertEquals(state, stateHandleProvider.getStateHandles().get(0).getState(null));
+
+		// Path created and is persistent
+		Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper);
+		assertNotNull(stat);
+		assertEquals(0, stat.getEphemeralOwner());
+
+		// Data is equal
+		@SuppressWarnings("unchecked")
+		Long actual = ((StateHandle<Long>) InstantiationUtil.deserializeObject(
+				ZooKeeper.getClient().getData().forPath(pathInZooKeeper),
+				ClassLoader.getSystemClassLoader())).getState(null);
+
+		assertEquals(state, actual);
+	}
+
+	/**
+	 * Tests that {@link CreateMode} is respected.
+	 */
+	@Test
+	public void testAddWithCreateMode() throws Exception {
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		Long state = 3457347234L;
+
+		CreateMode[] modes = CreateMode.values();
+		for (int i = 0; i < modes.length; i++) {
+			CreateMode mode = modes[i];
+			state += i;
+
+			String pathInZooKeeper = "/testAddWithCreateMode" + mode.name();
+
+			// Test
+			store.add(pathInZooKeeper, state, mode);
+
+			if (mode.isSequential()) {
+				// Figure out the sequential ID
+				List<String> paths = ZooKeeper.getClient().getChildren().forPath("/");
+				for (String p : paths) {
+					if (p.startsWith("testAddWithCreateMode" + mode.name())) {
+						pathInZooKeeper = "/" + p;
+						break;
+					}
+				}
+			}
+
+			// Verify
+			// State handle created
+			assertEquals(i + 1, stateHandleProvider.getStateHandles().size());
+			assertEquals(state, stateHandleProvider.getStateHandles().get(i).getState(null));
+
+			// Path created
+			Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper);
+
+			assertNotNull(stat);
+
+			// Is ephemeral or persistent
+			if (mode.isEphemeral()) {
+				assertTrue(stat.getEphemeralOwner() != 0);
+			}
+			else {
+				assertEquals(0, stat.getEphemeralOwner());
+			}
+
+			// Data is equal
+			@SuppressWarnings("unchecked")
+			Long actual = ((StateHandle<Long>) InstantiationUtil.deserializeObject(
+					ZooKeeper.getClient().getData().forPath(pathInZooKeeper),
+					ClassLoader.getSystemClassLoader())).getState(null);
+
+			assertEquals(state, actual);
+		}
+	}
+
+	/**
+	 * Tests that an existing path throws an Exception.
+	 */
+	@Test(expected = Exception.class)
+	public void testAddAlreadyExistingPath() throws Exception {
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		ZooKeeper.getClient().create().forPath("/testAddAlreadyExistingPath");
+
+		store.add("/testAddAlreadyExistingPath", 1L);
+	}
+
+	/**
+	 * Tests that the created state handle is discarded if ZooKeeper create fails.
+	 */
+	@Test
+	public void testAddDiscardStateHandleAfterFailure() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		CuratorFramework client = spy(ZooKeeper.getClient());
+		when(client.create()).thenThrow(new RuntimeException("Expected test Exception."));
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				client, stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testAddDiscardStateHandleAfterFailure";
+		final Long state = 81282227L;
+
+		try {
+			// Test
+			store.add(pathInZooKeeper, state);
+			fail("Did not throw expected exception");
+		}
+		catch (Exception ignored) {
+		}
+
+		// Verify
+		// State handle created and discarded
+		assertEquals(1, stateHandleProvider.getStateHandles().size());
+		assertEquals(state, stateHandleProvider.getStateHandles().get(0).getState(null));
+		assertEquals(1, stateHandleProvider.getStateHandles().get(0).getNumberOfDiscardCalls());
+	}
+
+	/**
+	 * Tests that a state handle is replaced.
+	 */
+	@Test
+	public void testReplace() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testReplace";
+		final Long initialState = 30968470898L;
+		final Long replaceState = 88383776661L;
+
+		// Test
+		store.add(pathInZooKeeper, initialState);
+		store.replace(pathInZooKeeper, 0, replaceState);
+
+		// Verify
+		// State handles created
+		assertEquals(2, stateHandleProvider.getStateHandles().size());
+		assertEquals(initialState, stateHandleProvider.getStateHandles().get(0).getState(null));
+		assertEquals(replaceState, stateHandleProvider.getStateHandles().get(1).getState(null));
+
+		// Path created and is persistent
+		Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper);
+		assertNotNull(stat);
+		assertEquals(0, stat.getEphemeralOwner());
+
+		// Data is equal
+		@SuppressWarnings("unchecked")
+		Long actual = ((StateHandle<Long>) InstantiationUtil.deserializeObject(
+				ZooKeeper.getClient().getData().forPath(pathInZooKeeper),
+				ClassLoader.getSystemClassLoader())).getState(null);
+
+		assertEquals(replaceState, actual);
+	}
+
+	/**
+	 * Tests that a non existing path throws an Exception.
+	 */
+	@Test(expected = Exception.class)
+	public void testReplaceNonExistingPath() throws Exception {
+		StateHandleProvider<Long> stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		store.replace("/testReplaceNonExistingPath", 0, 1L);
+	}
+
+	/**
+	 * Tests that the replace state handle is discarded if ZooKeeper setData fails.
+	 */
+	@Test
+	public void testReplaceDiscardStateHandleAfterFailure() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		CuratorFramework client = spy(ZooKeeper.getClient());
+		when(client.setData()).thenThrow(new RuntimeException("Expected test Exception."));
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				client, stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testReplaceDiscardStateHandleAfterFailure";
+		final Long initialState = 30968470898L;
+		final Long replaceState = 88383776661L;
+
+		// Test
+		store.add(pathInZooKeeper, initialState);
+
+		try {
+			store.replace(pathInZooKeeper, 0, replaceState);
+			fail("Did not throw expected exception");
+		}
+		catch (Exception ignored) {
+		}
+
+		// Verify
+		// State handle created and discarded
+		assertEquals(2, stateHandleProvider.getStateHandles().size());
+		assertEquals(initialState, stateHandleProvider.getStateHandles().get(0).getState(null));
+		assertEquals(replaceState, stateHandleProvider.getStateHandles().get(1).getState(null));
+		assertEquals(1, stateHandleProvider.getStateHandles().get(1).getNumberOfDiscardCalls());
+
+		// Initial value
+		@SuppressWarnings("unchecked")
+		Long actual = ((StateHandle<Long>) InstantiationUtil.deserializeObject(
+				ZooKeeper.getClient().getData().forPath(pathInZooKeeper),
+				ClassLoader.getSystemClassLoader())).getState(null);
+
+		assertEquals(initialState, actual);
+	}
+
+	/**
+	 * Tests get operation.
+	 */
+	@Test
+	public void testGetAndExists() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testGetAndExists";
+		final Long state = 311222268470898L;
+
+		// Test
+		assertEquals(-1, store.exists(pathInZooKeeper));
+
+		store.add(pathInZooKeeper, state);
+		StateHandle<Long> actual = store.get(pathInZooKeeper);
+
+		// Verify
+		assertEquals(state, actual.getState(null));
+		assertTrue(store.exists(pathInZooKeeper) >= 0);
+	}
+
+	/**
+	 * Tests that a non existing path throws an Exception.
+	 */
+	@Test(expected = Exception.class)
+	public void testGetNonExistingPath() throws Exception {
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		store.get("/testGetNonExistingPath");
+	}
+
+	/**
+	 * Tests that all added state is returned.
+	 */
+	@Test
+	public void testGetAll() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testGetAll";
+
+		final Set<Long> expected = new HashSet<>();
+		expected.add(311222268470898L);
+		expected.add(132812888L);
+		expected.add(27255442L);
+		expected.add(11122233124L);
+
+		// Test
+		for (long val : expected) {
+			store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL);
+		}
+
+		for (Tuple2<StateHandle<Long>, String> val : store.getAll()) {
+			assertTrue(expected.remove(val.f0.getState(null)));
+		}
+		assertEquals(0, expected.size());
+	}
+
+	/**
+	 * Tests that the state is returned sorted.
+	 */
+	@Test
+	public void testGetAllSortedByName() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testGetAllSortedByName";
+
+		final Long[] expected = new Long[] {
+				311222268470898L, 132812888L, 27255442L, 11122233124L };
+
+		// Test
+		for (long val : expected) {
+			store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL);
+		}
+
+		List<Tuple2<StateHandle<Long>, String>> actual = store.getAllSortedByName();
+		assertEquals(expected.length, actual.size());
+
+		for (int i = 0; i < expected.length; i++) {
+			assertEquals(expected[i], actual.get(i).f0.getState(null));
+		}
+	}
+
+	/**
+	 * Tests that state handles are correctly removed.
+	 */
+	@Test
+	public void testRemove() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testRemove";
+		final Long state = 27255442L;
+
+		store.add(pathInZooKeeper, state);
+
+		// Test
+		store.remove(pathInZooKeeper);
+
+		// Verify discarded
+		assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size());
+	}
+
+	/**
+	 * Tests that state handles are correctly removed with a callback.
+	 */
+	@Test
+	public void testRemoveWithCallback() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testRemoveWithCallback";
+		final Long state = 27255442L;
+
+		store.add(pathInZooKeeper, state);
+
+		final CountDownLatch sync = new CountDownLatch(1);
+		BackgroundCallback callback = mock(BackgroundCallback.class);
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				sync.countDown();
+				return null;
+			}
+		}).when(callback).processResult(eq(ZooKeeper.getClient()), any(CuratorEvent.class));
+
+		// Test
+		store.remove(pathInZooKeeper, callback);
+
+		// Verify discarded and callback called
+		assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size());
+
+		sync.await();
+
+		verify(callback, times(1))
+				.processResult(eq(ZooKeeper.getClient()), any(CuratorEvent.class));
+	}
+
+	/**
+	 * Tests that state handles are correctly discarded.
+	 */
+	@Test
+	public void testRemoveAndDiscardState() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testDiscard";
+		final Long state = 27255442L;
+
+		store.add(pathInZooKeeper, state);
+
+		// Test
+		store.removeAndDiscardState(pathInZooKeeper);
+
+		// Verify discarded
+		assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size());
+	}
+
+	/** Tests that all state handles are correctly discarded. */
+	@Test
+	public void testRemoveAndDiscardAllState() throws Exception {
+		// Setup
+		LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider();
+
+		ZooKeeperStateHandleStore<Long> store = new ZooKeeperStateHandleStore<>(
+				ZooKeeper.getClient(), stateHandleProvider);
+
+		// Config
+		final String pathInZooKeeper = "/testDiscardAll";
+
+		final Set<Long> expected = new HashSet<>();
+		expected.add(311222268470898L);
+		expected.add(132812888L);
+		expected.add(27255442L);
+		expected.add(11122233124L);
+
+		// Test
+		for (long val : expected) {
+			store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL);
+		}
+
+		store.removeAndDiscardAllState();
+
+		// Verify all discarded
+		assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size());
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Simple test helpers
+	// ---------------------------------------------------------------------------------------------
+
+	private static class LongStateHandleProvider implements StateHandleProvider<Long> {
+
+		private static final long serialVersionUID = 4572084854499402276L;
+
+		private final List<LongStateHandle> stateHandles = new ArrayList<>();
+
+		@Override
+		public StateHandle<Long> createStateHandle(Long state) {
+			LongStateHandle stateHandle = new LongStateHandle(state);
+			stateHandles.add(stateHandle);
+
+			return stateHandle;
+		}
+
+		public List<LongStateHandle> getStateHandles() {
+			return stateHandles;
+		}
+	}
+
+	private static class LongStateHandle implements StateHandle<Long> {
+
+		private static final long serialVersionUID = -3555329254423838912L;
+
+		private final Long state;
+
+		private int numberOfDiscardCalls;
+
+		public LongStateHandle(Long state) {
+			this.state = state;
+		}
+
+		@Override
+		public Long getState(ClassLoader ignored) throws Exception {
+			return state;
+		}
+
+		@Override
+		public void discardState() throws Exception {
+			numberOfDiscardCalls++;
+		}
+
+		public int getNumberOfDiscardCalls() {
+			return numberOfDiscardCalls;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
new file mode 100644
index 0000000..7ae89d1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.zookeeper;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.TestingServer;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+/**
+ * Simple ZooKeeper and CuratorFramework setup for tests.
+ */
+public class ZooKeeperTestEnvironment {
+
+	private final TestingServer zooKeeperServer;
+
+	private final TestingCluster zooKeeperCluster;
+
+	private final CuratorFramework client;
+
+	/**
+	 * Starts a ZooKeeper cluster with the number of quorum peers and a client.
+	 *
+	 * @param numberOfZooKeeperQuorumPeers Starts a {@link TestingServer}, if <code>1</code>.
+	 *                                     Starts a {@link TestingCluster}, if <code>=>1</code>.
+	 */
+	public ZooKeeperTestEnvironment(int numberOfZooKeeperQuorumPeers) {
+		if (numberOfZooKeeperQuorumPeers <= 0) {
+			throw new IllegalArgumentException("Number of peers needs to be >= 1.");
+		}
+
+		final Configuration conf = new Configuration();
+
+		try {
+			if (numberOfZooKeeperQuorumPeers == 1) {
+				zooKeeperServer = new TestingServer(true);
+				zooKeeperCluster = null;
+
+				conf.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY,
+						zooKeeperServer.getConnectString());
+			}
+			else {
+				zooKeeperServer = null;
+				zooKeeperCluster = new TestingCluster(numberOfZooKeeperQuorumPeers);
+
+				zooKeeperCluster.start();
+
+				conf.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY,
+						zooKeeperCluster.getConnectString());
+			}
+
+			client = ZooKeeperUtils.startCuratorFramework(conf);
+
+			client.newNamespaceAwareEnsurePath("/")
+					.ensure(client.getZookeeperClient());
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Error setting up ZooKeeperTestEnvironment", e);
+		}
+	}
+
+	/**
+	 * Shutdown the client and ZooKeeper server/cluster.
+	 */
+	public void shutdown() throws Exception {
+		if (client != null) {
+			client.close();
+		}
+
+		if (zooKeeperServer != null) {
+			zooKeeperServer.close();
+		}
+
+		if (zooKeeperCluster != null) {
+			zooKeeperCluster.close();
+		}
+	}
+
+	public String getConnectString() {
+		if (zooKeeperServer != null) {
+			return zooKeeperServer.getConnectString();
+		}
+		else {
+			return zooKeeperCluster.getConnectString();
+		}
+	}
+
+	/**
+	 * Returns a client for the started ZooKeeper server/cluster.
+	 */
+	public CuratorFramework getClient() {
+		return client;
+	}
+
+	/**
+	 * Creates a new client for the started ZooKeeper server/cluster.
+	 */
+	public CuratorFramework createClient() {
+		Configuration config = new Configuration();
+		config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, getConnectString());
+		return ZooKeeperUtils.startCuratorFramework(config);
+	}
+
+	/**
+	 * Deletes all ZNodes under the root node.
+	 *
+	 * @throws Exception If the ZooKeeper operation fails
+	 */
+	public void deleteAll() throws Exception {
+		final String path = "/" + client.getNamespace();
+		ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
index d1b8fac..9a1cde0 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
@@ -18,9 +18,12 @@
 
 package org.apache.flink.runtime.executiongraph
 
+import java.util.concurrent.TimeUnit
+
 import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway
 import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, JobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks
@@ -32,6 +35,7 @@ import org.scalatest.junit.JUnitRunner
 import org.scalatest.{Matchers, WordSpecLike}
 
 import scala.collection.JavaConverters._
+import scala.concurrent.duration.FiniteDuration
 
 @RunWith(classOf[JUnitRunner])
 class ExecutionGraphRestartTest extends WordSpecLike with Matchers {
@@ -126,8 +130,23 @@ class ExecutionGraphRestartTest extends WordSpecLike with Matchers {
         for (vertex <- eg.getAllExecutionVertices.asScala) {
           vertex.getCurrentExecutionAttempt().cancelingComplete()
         }
-        
+
+        val timeout = new FiniteDuration(2, TimeUnit.MINUTES)
+
+        // Wait for async restart
+        var deadline = timeout.fromNow
+        while (deadline.hasTimeLeft() && eg.getState != JobStatus.RUNNING) {
+          Thread.sleep(100)
+        }
+
         eg.getState should equal(JobStatus.RUNNING)
+
+        // Wait for deploying after async restart
+        deadline = timeout.fromNow
+        while (deadline.hasTimeLeft() && eg.getAllExecutionVertices.asScala.exists(
+          _.getCurrentExecutionAttempt.getState != ExecutionState.DEPLOYING)) {
+          Thread.sleep(100)
+        }
         
         for (vertex <- eg.getAllExecutionVertices.asScala) {
           vertex.getCurrentExecutionAttempt().markFinished()

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
index c9ae1e4..703d7bf 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
@@ -43,28 +43,28 @@ import scala.concurrent.{Await, Future}
  *                          otherwise false
  */
 class TestingCluster(
-    userConfiguration: Configuration,
-    singleActorSystem: Boolean,
-    synchronousDispatcher: Boolean,
-    streamingMode: StreamingMode)
+                      userConfiguration: Configuration,
+                      singleActorSystem: Boolean,
+                      synchronousDispatcher: Boolean,
+                      streamingMode: StreamingMode)
   extends FlinkMiniCluster(
     userConfiguration,
     singleActorSystem,
     streamingMode) {
-  
+
 
   def this(userConfiguration: Configuration,
            singleActorSystem: Boolean,
            synchronousDispatcher: Boolean)
-       = this(userConfiguration, singleActorSystem, synchronousDispatcher, StreamingMode.BATCH_ONLY)
+  = this(userConfiguration, singleActorSystem, synchronousDispatcher, StreamingMode.BATCH_ONLY)
 
   def this(userConfiguration: Configuration, singleActorSystem: Boolean)
-       = this(userConfiguration, singleActorSystem, false)
+  = this(userConfiguration, singleActorSystem, false)
 
   def this(userConfiguration: Configuration) = this(userConfiguration, true, false)
-  
+
   // --------------------------------------------------------------------------
-  
+
   override def generateConfiguration(userConfig: Configuration): Configuration = {
     val cfg = new Configuration()
     cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost")
@@ -100,16 +100,18 @@ class TestingCluster(
     }
 
     val (executionContext,
-      instanceManager,
-      scheduler,
-      libraryCacheManager,
-      executionRetries,
-      delayBetweenRetries,
-      timeout,
-      archiveCount,
-      leaderElectionService) = JobManager.createJobManagerComponents(
-        config,
-        createLeaderElectionService())
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    executionRetries,
+    delayBetweenRetries,
+    timeout,
+    archiveCount,
+    leaderElectionService,
+    submittedJobsGraphs,
+    checkpointRecoveryFactory) = JobManager.createJobManagerComponents(
+      config,
+      createLeaderElectionService())
 
     val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount))
     val archive = actorSystem.actorOf(testArchiveProps, archiveName)
@@ -126,7 +128,9 @@ class TestingCluster(
         delayBetweenRetries,
         timeout,
         streamingMode,
-        leaderElectionService))
+        leaderElectionService,
+        submittedJobsGraphs,
+        checkpointRecoveryFactory))
 
     val dispatcherJobManagerProps = if (synchronousDispatcher) {
       // disable asynchronous futures (e.g. accumulator update in Heartbeat)

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
index 312a1e5..be72003 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
@@ -18,32 +18,18 @@
 
 package org.apache.flink.runtime.testingUtils
 
-import akka.actor.{Cancellable, Terminated, ActorRef}
-import akka.pattern.pipe
-import akka.pattern.ask
-import org.apache.flink.api.common.JobID
+import akka.actor.ActorRef
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.{StreamingMode, FlinkActor}
 import org.apache.flink.runtime.StreamingMode
-import org.apache.flink.runtime.execution.ExecutionState
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.instance.InstanceManager
-import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.JobManager
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
-import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
-import org.apache.flink.runtime.messages.JobManagerMessages.GrantLeadership
-import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
-import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
-import org.apache.flink.runtime.testingUtils.TestingMessages.{CheckIfJobRemoved, Alive,
-DisableDisconnect}
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged
+import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
 
-import scala.concurrent.{ExecutionContext, Future}
+import scala.concurrent.ExecutionContext
 import scala.concurrent.duration._
-
 import scala.language.postfixOps
 
 /** JobManager implementation extended by testing messages
@@ -70,7 +56,9 @@ class TestingJobManager(
     delayBetweenRetries: Long,
     timeout: FiniteDuration,
     mode: StreamingMode,
-    leaderElectionService: LeaderElectionService)
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory)
   extends JobManager(
     flinkConfiguration,
     executionContext,
@@ -82,5 +70,7 @@ class TestingJobManager(
     delayBetweenRetries,
     timeout,
     mode,
-    leaderElectionService)
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory)
   with TestingJobManagerLike {}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index b607433..72a8c25 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.StatefulTask;
+import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -541,7 +542,6 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
 					}
 			}
 		}
-	}
 
 	/**
 	 * Registers a timer.

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
index c7f7698..11eb174 100644
--- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
+++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
@@ -20,22 +20,20 @@ package org.apache.flink.test.util
 
 import java.util.concurrent.TimeoutException
 
-import akka.pattern.ask
-import akka.actor.{Props, ActorRef, ActorSystem}
+import akka.actor.{ActorRef, ActorSystem}
 import akka.pattern.Patterns._
+import akka.pattern.ask
 import org.apache.curator.test.TestingCluster
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.jobmanager.{RecoveryMode, JobManager}
+import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode}
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
 import org.apache.flink.runtime.taskmanager.TaskManager
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages
-.NotifyWhenRegisteredAtJobManager
-import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingTaskManager,
-TestingJobManager, TestingMemoryArchivist}
+import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager
+import org.apache.flink.runtime.testingUtils.{TestingJobManager, TestingMemoryArchivist, TestingTaskManager, TestingUtils}
 
-import scala.concurrent.{Future, Await}
+import scala.concurrent.{Await, Future}
 
 /**
  * A forkable mini cluster is a special case of the mini cluster, used for parallel test execution
@@ -47,20 +45,20 @@ import scala.concurrent.{Future, Await}
  *                          same [[ActorSystem]], otherwise false.
  */
 class ForkableFlinkMiniCluster(
-    userConfiguration: Configuration,
-    singleActorSystem: Boolean,
-    streamingMode: StreamingMode)
+                                userConfiguration: Configuration,
+                                singleActorSystem: Boolean,
+                                streamingMode: StreamingMode)
   extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) {
 
-  def this(userConfiguration: Configuration, singleActorSystem: Boolean) 
-       = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY)
+  def this(userConfiguration: Configuration, singleActorSystem: Boolean)
+  = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY)
 
   def this(userConfiguration: Configuration) = this(userConfiguration, true)
-  
+
   // --------------------------------------------------------------------------
 
   var zookeeperCluster: Option[TestingCluster] = None
-  
+
   override def generateConfiguration(userConfiguration: Configuration): Configuration = {
     val forNumberString = System.getProperty("forkNumber")
 
@@ -264,10 +262,10 @@ object ForkableFlinkMiniCluster {
   import org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT
 
   def startCluster(
-      numSlots: Int,
-      numTaskManagers: Int,
-      timeout: String = DEFAULT_AKKA_ASK_TIMEOUT)
-    : ForkableFlinkMiniCluster = {
+                    numSlots: Int,
+                    numTaskManagers: Int,
+                    timeout: String = DEFAULT_AKKA_ASK_TIMEOUT)
+  : ForkableFlinkMiniCluster = {
 
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)


[28/47] flink git commit: [FLINK-2852] [test-stability] Fix ScalaShellITSuite and ScalaShellLocalStartupITCase

Posted by tr...@apache.org.
[FLINK-2852] [test-stability] Fix ScalaShellITSuite and ScalaShellLocalStartupITCase

Changes test program to use an int accumulator which is checked at the end of the program.
This avoids to look for the "Job status changed to FINISHED." string in the stdout output,
which can sometimes not be printed if the JobExecutionResult arrives earlier than the
JobStatusChanged message at the JobClientActor.


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

Branch: refs/heads/master
Commit: 630798d36f0e9cf0b7b6139ccfd8583ba0ae80b1
Parents: e3ad962
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Oct 15 02:07:53 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:53 2015 +0200

----------------------------------------------------------------------
 flink-staging/flink-scala-shell/pom.xml         |  33 --
 .../org/apache/flink/api/scala/FlinkILoop.scala |   1 -
 .../org/apache/flink/api/scala/FlinkShell.scala |  38 ++-
 .../flink/api/scala/ScalaShellITCase.scala      | 332 +++++++++++++++++++
 .../flink/api/scala/ScalaShellITSuite.scala     | 286 ----------------
 .../scala/ScalaShellLocalStartupITCase.scala    |  75 +++--
 tools/log4j-travis.properties                   |   1 -
 7 files changed, 403 insertions(+), 363 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/pom.xml b/flink-staging/flink-scala-shell/pom.xml
index c4b5bd6..371257b 100644
--- a/flink-staging/flink-scala-shell/pom.xml
+++ b/flink-staging/flink-scala-shell/pom.xml
@@ -195,39 +195,6 @@ under the License.
 			</plugin>
 
 			<plugin>
-				<groupId>org.scalatest</groupId>
-				<artifactId>scalatest-maven-plugin</artifactId>
-				<version>1.0</version>
-				<configuration>
-					<reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
-					<stdout>W</stdout> <!-- Skip coloring output -->
-				</configuration>
-				<executions>
-					<execution>
-						<id>scala-test</id>
-						<goals>
-							<goal>test</goal>
-						</goals>
-						<configuration>
-							<suffixes>(?&lt;!(IT|Integration))(Test|Suite|Case)</suffixes>
-							<argLine>-Xms256m -Xmx800m -Dlog4j.configuration=${log4j.configuration} -Dlog.dir=${log.dir} -Dmvn.forkNumber=1 -XX:-UseGCOverheadLimit</argLine>
-						</configuration>
-					</execution>
-					<execution>
-						<id>integration-test</id>
-						<phase>integration-test</phase>
-						<goals>
-							<goal>test</goal>
-						</goals>
-						<configuration>
-							<suffixes>(IT|Integration)(Test|Suite|Case)</suffixes>
-							<argLine>-Xms256m -Xmx800m -Dlog4j.configuration=${log4j.configuration} -Dlog.dir=${log.dir} -Dmvn.forkNumber=1 -XX:-UseGCOverheadLimit</argLine>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
 				<groupId>org.scalastyle</groupId>
 				<artifactId>scalastyle-maven-plugin</artifactId>
 				<version>0.5.0</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala b/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala
index 9fb45a8..a26ac2e 100644
--- a/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala
+++ b/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala
@@ -233,6 +233,5 @@ HINT: You can use print() on a DataSet to print the contents to this shell.
   }
 
   def getExternalJars(): Array[String] = externalJars.getOrElse(Array.empty[String])
-
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 54bbf80..eb7f816 100644
--- a/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-staging/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -22,7 +22,7 @@ import java.io.{StringWriter, BufferedReader}
 
 import org.apache.flink.api.common.ExecutionMode
 
-import org.apache.flink.configuration.Configuration
+import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
 
 import scala.tools.nsc.Settings
@@ -86,7 +86,7 @@ object FlinkShell {
           config.flinkShellExecutionMode,
           config.externalJars)
 
-      case _ => println("Could not parse program arguments")
+      case _ => System.out.println("Could not parse program arguments")
     }
   }
 
@@ -97,36 +97,41 @@ object FlinkShell {
       executionMode: ExecutionMode.Value,
       externalJars: Option[Array[String]] = None): Unit ={
     
-    println("Starting Flink Shell:")
+    System.out.println("Starting Flink Shell:")
 
     // either port or userhost not specified by user, create new minicluster
     val (host: String, port: Int, cluster: Option[LocalFlinkMiniCluster]) =
       executionMode match {
         case ExecutionMode.LOCAL =>
-          val miniCluster = new LocalFlinkMiniCluster(new Configuration, false)
+          val config = new Configuration()
+          config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
+          val miniCluster = new LocalFlinkMiniCluster(config, false)
           miniCluster.start()
           val port = miniCluster.getLeaderRPCPort
-          println(s"\nStarting local Flink cluster (host: localhost, port: $port).\n")
+          System.out.println(s"\nStarting local Flink cluster (host: localhost, port: $port).\n")
           ("localhost", port, Some(miniCluster))
 
         case ExecutionMode.REMOTE =>
           if (userHost == "none" || userPort == -1) {
-            println("Error: <host> or <port> not specified!")
+            System.out.println("Error: <host> or <port> not specified!")
             return
           } else {
-            println(s"\nConnecting to Flink cluster (host: $userHost, port: $userPort).\n")
+            System.out.println(
+              s"\nConnecting to Flink cluster (host: $userHost, port: $userPort).\n")
             (userHost, userPort, None)
           }
 
         case ExecutionMode.UNDEFINED =>
-          println("Error: please specify execution mode:")
-          println("[local | remote <host> <port>]")
+          System.out.println("Error: please specify execution mode:")
+          System.out.println("[local | remote <host> <port>]")
           return
       }
 
+    var repl: Option[FlinkILoop] = None
+
     try {
       // custom shell
-      val repl: FlinkILoop =
+      repl = Some(
         bufferedReader match {
 
           case Some(br) =>
@@ -135,21 +140,20 @@ object FlinkShell {
 
           case None =>
             new FlinkILoop(host, port, externalJars)
-        }
+        })
 
       val settings = new Settings()
 
       settings.usejavacp.value = true
+      settings.Yreplsync.value = true
 
       // start scala interpreter shell
-      repl.process(settings)
+      repl.foreach(_.process(settings))
     } finally {
-      cluster match {
-        case Some(c) => c.stop()
-        case None =>
-      }
+      repl.foreach(_.closeInterpreter())
+      cluster.foreach(_.stop())
     }
 
-    println(" good bye ..")
+    System.out.println(" good bye ..")
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
new file mode 100644
index 0000000..de2f3ec
--- /dev/null
+++ b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
@@ -0,0 +1,332 @@
+/*
+ * 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.api.scala
+
+import java.io._
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.runtime.StreamingMode
+import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils}
+import org.apache.flink.util.TestLogger
+import org.junit.{AfterClass, BeforeClass, Test, Assert}
+
+import scala.concurrent.duration.FiniteDuration
+import scala.tools.nsc.Settings
+
+class ScalaShellITCase extends TestLogger {
+
+  import ScalaShellITCase._
+
+  /** Prevent re-creation of environment */
+  @Test
+  def testPreventRecreation(): Unit = {
+
+    val input: String =
+      """
+        val env = ExecutionEnvironment.getExecutionEnvironment
+      """.stripMargin
+
+    val output: String = processInShell(input)
+
+    Assert.assertTrue(output.contains(
+      "UnsupportedOperationException: Execution Environment is already " +
+      "defined for this shell"))
+  }
+
+  /** Iteration test with iterative Pi example */
+  @Test
+  def testIterativePI(): Unit = {
+
+    val input: String =
+      """
+        val initial = env.fromElements(0)
+        val count = initial.iterate(10000) { iterationInput: DataSet[Int] =>
+          val result = iterationInput.map { i =>
+            val x = Math.random()
+            val y = Math.random()
+            i + (if (x * x + y * y < 1) 1 else 0)
+          }
+          result
+        }
+        val result = count map { c => c / 10000.0 * 4 }
+        result.collect()
+      """.stripMargin
+
+    val output: String = processInShell(input)
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("error"))
+    Assert.assertFalse(output.contains("Exception"))
+  }
+
+  /** WordCount in Shell */
+  @Test
+  def testWordCount(): Unit = {
+    val input =
+      """
+        val text = env.fromElements("To be, or not to be,--that is the question:--",
+        "Whether 'tis nobler in the mind to suffer",
+        "The slings and arrows of outrageous fortune",
+        "Or to take arms against a sea of troubles,")
+        val counts = text.flatMap { _.toLowerCase.split("\\W+") }.map { (_, 1) }.groupBy(0).sum(1)
+        val result = counts.print()
+      """.stripMargin
+
+    val output = processInShell(input)
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("error"))
+    Assert.assertFalse(output.contains("Exception"))
+
+    // some of the words that should be included
+    Assert.assertTrue(output.contains("(a,1)"))
+    Assert.assertTrue(output.contains("(whether,1)"))
+    Assert.assertTrue(output.contains("(to,4)"))
+    Assert.assertTrue(output.contains("(arrows,1)"))
+  }
+
+  /** Sum 1..10, should be 55 */
+  @Test
+  def testSum: Unit = {
+    val input =
+      """
+        val input: DataSet[Int] = env.fromElements(0,1,2,3,4,5,6,7,8,9,10)
+        val reduced = input.reduce(_+_)
+        reduced.print
+      """.stripMargin
+
+    val output = processInShell(input)
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("error"))
+    Assert.assertFalse(output.contains("Exception"))
+
+    Assert.assertTrue(output.contains("55"))
+  }
+
+  /** WordCount in Shell with custom case class */
+  @Test
+  def testWordCountWithCustomCaseClass: Unit = {
+    val input =
+      """
+      case class WC(word: String, count: Int)
+      val wordCounts = env.fromElements(
+        new WC("hello", 1),
+        new WC("world", 2),
+        new WC("world", 8))
+      val reduced = wordCounts.groupBy(0).sum(1)
+      reduced.print()
+      """.stripMargin
+
+    val output = processInShell(input)
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("error"))
+    Assert.assertFalse(output.contains("Exception"))
+
+    Assert.assertTrue(output.contains("WC(hello,1)"))
+    Assert.assertTrue(output.contains("WC(world,10)"))
+  }
+
+  /** Submit external library */
+  @Test
+  def testSubmissionOfExternalLibrary: Unit = {
+    val input =
+      """
+        import org.apache.flink.ml.math._
+        val denseVectors = env.fromElements(DenseVector(1.0, 2.0, 3.0))
+        denseVectors.print()
+      """.stripMargin
+
+    // find jar file that contains the ml code
+    var externalJar = ""
+    val folder = new File("../flink-ml/target/")
+    val listOfFiles = folder.listFiles()
+
+    for (i <- listOfFiles.indices) {
+      val filename: String = listOfFiles(i).getName
+      if (!filename.contains("test") && !filename.contains("original") && filename.contains(
+        ".jar")) {
+        externalJar = listOfFiles(i).getAbsolutePath
+      }
+    }
+
+    assert(externalJar != "")
+
+    val output: String = processInShell(input, Option(externalJar))
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("error"))
+    Assert.assertFalse(output.contains("Exception"))
+
+    Assert.assertTrue(output.contains("\nDenseVector(1.0, 2.0, 3.0)"))
+  }
+
+
+  /**
+   * tests flink shell startup with remote cluster (starts cluster internally)
+   */
+  @Test
+  def testRemoteCluster: Unit = {
+
+    val input: String =
+      """
+        |import org.apache.flink.api.common.functions.RichMapFunction
+        |import org.apache.flink.api.java.io.PrintingOutputFormat
+        |import org.apache.flink.api.common.accumulators.IntCounter
+        |import org.apache.flink.configuration.Configuration
+        |
+        |val els = env.fromElements("foobar","barfoo")
+        |val mapped = els.map{
+        | new RichMapFunction[String, String]() {
+        |   var intCounter: IntCounter = _
+        |   override def open(conf: Configuration): Unit = {
+        |     intCounter = getRuntimeContext.getIntCounter("intCounter")
+        |   }
+        |
+        |   def map(element: String): String = {
+        |     intCounter.add(1)
+        |     element
+        |   }
+        | }
+        |}
+        |mapped.output(new PrintingOutputFormat())
+        |val executionResult = env.execute("Test Job")
+        |System.out.println("IntCounter: " + executionResult.getIntCounterResult("intCounter"))
+        |
+        |:q
+      """.stripMargin
+
+    val in: BufferedReader = new BufferedReader(
+      new StringReader(
+        input + "\n"))
+    val out: StringWriter = new StringWriter
+
+    val baos: ByteArrayOutputStream = new ByteArrayOutputStream
+    val oldOut: PrintStream = System.out
+    System.setOut(new PrintStream(baos))
+
+    val (c, args) = cluster match{
+      case Some(cl) =>
+        val arg = Array("remote",
+          cl.hostname,
+          Integer.toString(cl.getLeaderRPCPort))
+        (cl, arg)
+      case None =>
+        throw new AssertionError("Cluster creation failed.")
+    }
+
+    //start scala shell with initialized
+    // buffered reader for testing
+    FlinkShell.bufferedReader = Some(in)
+    FlinkShell.main(args)
+    baos.flush()
+
+    val output: String = baos.toString
+    System.setOut(oldOut)
+
+    Assert.assertTrue(output.contains("IntCounter: 2"))
+    Assert.assertTrue(output.contains("foobar"))
+    Assert.assertTrue(output.contains("barfoo"))
+
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("Error"))
+    Assert.assertFalse(output.contains("ERROR"))
+    Assert.assertFalse(output.contains("Exception"))
+  }
+}
+
+object ScalaShellITCase {
+  var cluster: Option[ForkableFlinkMiniCluster] = None
+  val parallelism = 4
+
+  @BeforeClass
+  def beforeAll(): Unit = {
+    val cl = TestBaseUtils.startCluster(
+      1,
+      parallelism,
+      StreamingMode.BATCH_ONLY,
+      false,
+      false,
+      false)
+
+    cluster = Some(cl)
+  }
+
+  @AfterClass
+  def afterAll(): Unit = {
+    // The Scala interpreter somehow changes the class loader. Therfore, we have to reset it
+    Thread.currentThread().setContextClassLoader(classOf[ScalaShellITCase].getClassLoader)
+    cluster.foreach(c => TestBaseUtils.stopCluster(c, new FiniteDuration(1000, TimeUnit.SECONDS)))
+  }
+
+  /**
+   * Run the input using a Scala Shell and return the output of the shell.
+   * @param input commands to be processed in the shell
+   * @return output of shell
+   */
+  def processInShell(input: String, externalJars: Option[String] = None): String = {
+    val in = new BufferedReader(new StringReader(input + "\n"))
+    val out = new StringWriter()
+    val baos = new ByteArrayOutputStream()
+
+    val oldOut = System.out
+    System.setOut(new PrintStream(baos))
+
+    // new local cluster
+    val host = "localhost"
+    val port = cluster match {
+      case Some(c) => c.getLeaderRPCPort
+      case _ => throw new RuntimeException("Test cluster not initialized.")
+    }
+
+    val repl = externalJars match {
+      case Some(ej) => new FlinkILoop(
+        host, port,
+        Option(Array(ej)),
+        in, new PrintWriter(out))
+
+      case None => new FlinkILoop(
+        host, port,
+        in, new PrintWriter(out))
+    }
+
+    repl.settings = new Settings()
+
+    // enable this line to use scala in intellij
+    repl.settings.usejavacp.value = true
+
+    externalJars match {
+      case Some(ej) => repl.settings.classpath.value = ej
+      case None =>
+    }
+
+    repl.process(repl.settings)
+
+    repl.closeInterpreter()
+
+    System.setOut(oldOut)
+
+    baos.flush()
+
+    val stdout = baos.toString
+
+    out.toString + stdout
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
deleted file mode 100644
index c8b1990..0000000
--- a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.scala
-
-import java.io._
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.runtime.StreamingMode
-import org.apache.flink.test.util.{TestEnvironment, ForkableFlinkMiniCluster, TestBaseUtils}
-import org.junit.runner.RunWith
-import org.scalatest.junit.JUnitRunner
-import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
-
-import scala.concurrent.duration.FiniteDuration
-import scala.tools.nsc.Settings
-
-@RunWith(classOf[JUnitRunner])
-class ScalaShellITSuite extends FunSuite with Matchers with BeforeAndAfterAll {
-
-  var cluster: Option[ForkableFlinkMiniCluster] = None
-  val parallelism = 4
-
-  test("Prevent re-creation of environment") {
-
-    val input: String =
-      """
-        val env = ExecutionEnvironment.getExecutionEnvironment
-      """.stripMargin
-
-    val output: String = processInShell(input)
-
-    output should include("UnsupportedOperationException: Execution Environment is already " +
-      "defined for this shell")
-  }
-
-  test("Iteration test with iterative Pi example") {
-
-    val input: String =
-      """
-        val initial = env.fromElements(0)
-        val count = initial.iterate(10000) { iterationInput: DataSet[Int] =>
-          val result = iterationInput.map { i =>
-            val x = Math.random()
-            val y = Math.random()
-            i + (if (x * x + y * y < 1) 1 else 0)
-          }
-          result
-        }
-        val result = count map { c => c / 10000.0 * 4 }
-        result.collect()
-      """.stripMargin
-
-    val output: String = processInShell(input)
-
-    output should not include "failed"
-    output should not include "error"
-    output should not include "Exception"
-  }
-
-  test("WordCount in Shell") {
-    val input =
-      """
-        val text = env.fromElements("To be, or not to be,--that is the question:--",
-        "Whether 'tis nobler in the mind to suffer",
-        "The slings and arrows of outrageous fortune",
-        "Or to take arms against a sea of troubles,")
-        val counts = text.flatMap { _.toLowerCase.split("\\W+") }.map { (_, 1) }.groupBy(0).sum(1)
-        val result = counts.print()
-      """.stripMargin
-
-    val output = processInShell(input)
-
-    output should not include "failed"
-    output should not include "error"
-    output should not include "Exception"
-
-    // some of the words that should be included
-    output should include("(a,1)")
-    output should include("(whether,1)")
-    output should include("(to,4)")
-    output should include("(arrows,1)")
-  }
-
-  test("Sum 1..10, should be 55") {
-    val input =
-      """
-        val input: DataSet[Int] = env.fromElements(0,1,2,3,4,5,6,7,8,9,10)
-        val reduced = input.reduce(_+_)
-        reduced.print
-      """.stripMargin
-
-    val output = processInShell(input)
-
-    output should not include "failed"
-    output should not include "error"
-    output should not include "Exception"
-
-    output should include("55")
-  }
-
-  test("WordCount in Shell with custom case class") {
-    val input =
-      """
-      case class WC(word: String, count: Int)
-      val wordCounts = env.fromElements(
-        new WC("hello", 1),
-        new WC("world", 2),
-        new WC("world", 8))
-      val reduced = wordCounts.groupBy(0).sum(1)
-      reduced.print()
-      """.stripMargin
-
-    val output = processInShell(input)
-
-    output should not include "failed"
-    output should not include "error"
-    output should not include "Exception"
-
-    output should include("WC(hello,1)")
-    output should include("WC(world,10)")
-  }
-
-  test("Submit external library") {
-    val input =
-      """
-        import org.apache.flink.ml.math._
-        val denseVectors = env.fromElements(DenseVector(1.0, 2.0, 3.0))
-        denseVectors.print()
-      """.stripMargin
-
-    // find jar file that contains the ml code
-    var externalJar = ""
-    val folder = new File("../flink-ml/target/")
-    val listOfFiles = folder.listFiles()
-
-    for (i <- listOfFiles.indices) {
-      val filename: String = listOfFiles(i).getName
-      if (!filename.contains("test") && !filename.contains("original") && filename.contains(
-        ".jar")) {
-        println("ive found file:" + listOfFiles(i).getAbsolutePath)
-        externalJar = listOfFiles(i).getAbsolutePath
-      }
-    }
-
-    assert(externalJar != "")
-
-    val output: String = processInShell(input, Option(externalJar))
-
-    output should not include "failed"
-    output should not include "error"
-    output should not include "Exception"
-
-    output should include("\nDenseVector(1.0, 2.0, 3.0)")
-  }
-
-  /**
-   * Run the input using a Scala Shell and return the output of the shell.
-   * @param input commands to be processed in the shell
-   * @return output of shell
-   */
-  def processInShell(input: String, externalJars: Option[String] = None): String = {
-    val in = new BufferedReader(new StringReader(input + "\n"))
-    val out = new StringWriter()
-    val baos = new ByteArrayOutputStream()
-
-    val oldOut = System.out
-    System.setOut(new PrintStream(baos))
-
-    // new local cluster
-    val host = "localhost"
-    val port = cluster match {
-      case Some(c) => c.getLeaderRPCPort
-      case _ => throw new RuntimeException("Test cluster not initialized.")
-    }
-
-    val repl = externalJars match {
-      case Some(ej) => new FlinkILoop(
-        host, port,
-        Option(Array(ej)),
-        in, new PrintWriter(out))
-
-      case None => new FlinkILoop(
-        host, port,
-        in, new PrintWriter(out))
-    }
-
-    repl.settings = new Settings()
-
-    // enable this line to use scala in intellij
-    repl.settings.usejavacp.value = true
-
-    externalJars match {
-      case Some(ej) => repl.settings.classpath.value = ej
-      case None =>
-    }
-
-    repl.process(repl.settings)
-
-    repl.closeInterpreter()
-
-    System.setOut(oldOut)
-
-    baos.flush()
-
-    val stdout = baos.toString
-
-    out.toString + stdout
-  }
-
-  /**
-   * tests flink shell startup with remote cluster (starts cluster internally)
-   */
-  test("start flink scala shell with remote cluster") {
-
-    val input: String = "val els = env.fromElements(\"a\",\"b\");\n" +
-      "els.print\nError\n:q\n"
-
-    val in: BufferedReader = new BufferedReader(
-      new StringReader(
-        input + "\n"))
-    val out: StringWriter = new StringWriter
-
-    val baos: ByteArrayOutputStream = new ByteArrayOutputStream
-    val oldOut: PrintStream = System.out
-    System.setOut(new PrintStream(baos))
-
-    val (c, args) = cluster match{
-      case Some(cl) =>
-        val arg = Array("remote",
-          cl.hostname,
-          Integer.toString(cl.getLeaderRPCPort))
-        (cl, arg)
-      case None =>
-        fail("Cluster creation failed!")
-    }
-
-    //start scala shell with initialized
-    // buffered reader for testing
-    FlinkShell.bufferedReader = Some(in)
-    FlinkShell.main(args)
-    baos.flush()
-
-    val output: String = baos.toString
-    System.setOut(oldOut)
-
-    output should include("Job execution switched to status FINISHED.")
-    output should include("a\nb")
-
-    output should not include "Error"
-    output should not include "ERROR"
-    output should not include "Exception"
-    output should not include "failed"
-  }
-
-  override def beforeAll(): Unit = {
-    val cl = TestBaseUtils.startCluster(
-      1,
-      parallelism,
-      StreamingMode.BATCH_ONLY,
-      false,
-      false,
-      false)
-
-    cluster = Some(cl)
-  }
-
-  override def afterAll(): Unit = {
-    cluster.foreach(c => TestBaseUtils.stopCluster(c, new FiniteDuration(1000, TimeUnit.SECONDS)))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala
index 60da09e..57bbd9b 100644
--- a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala
+++ b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala
@@ -20,20 +20,44 @@ package org.apache.flink.api.scala
 
 import java.io._
 
-import org.junit.runner.RunWith
-import org.scalatest.{Matchers, FunSuite}
-import org.scalatest.junit.JUnitRunner
+import org.apache.flink.util.TestLogger
+import org.junit.Test
+import org.junit.Assert
 
+class ScalaShellLocalStartupITCase extends TestLogger {
 
-@RunWith(classOf[JUnitRunner])
-class ScalaShellLocalStartupITCase extends FunSuite with Matchers {
-
-    /**
-     * tests flink shell with local setup through startup script in bin folder
-     */
-    test("start flink scala shell with local cluster") {
-
-      val input: String = "val els = env.fromElements(\"a\",\"b\");\n" + "els.print\nError\n:q\n"
+  /**
+   * tests flink shell with local setup through startup script in bin folder
+   */
+  @Test
+  def testLocalCluster: Unit = {
+    val input: String =
+      """
+        |import org.apache.flink.api.common.functions.RichMapFunction
+        |import org.apache.flink.api.java.io.PrintingOutputFormat
+        |import org.apache.flink.api.common.accumulators.IntCounter
+        |import org.apache.flink.configuration.Configuration
+        |
+        |val els = env.fromElements("foobar","barfoo")
+        |val mapped = els.map{
+        | new RichMapFunction[String, String]() {
+        |   var intCounter: IntCounter = _
+        |   override def open(conf: Configuration): Unit = {
+        |     intCounter = getRuntimeContext.getIntCounter("intCounter")
+        |   }
+        |
+        |   def map(element: String): String = {
+        |     intCounter.add(1)
+        |     element
+        |   }
+        | }
+        |}
+        |mapped.output(new PrintingOutputFormat())
+        |val executionResult = env.execute("Test Job")
+        |System.out.println("IntCounter: " + executionResult.getIntCounterResult("intCounter"))
+        |
+        |:q
+      """.stripMargin
       val in: BufferedReader = new BufferedReader(new StringReader(input + "\n"))
       val out: StringWriter = new StringWriter
       val baos: ByteArrayOutputStream = new ByteArrayOutputStream
@@ -41,20 +65,21 @@ class ScalaShellLocalStartupITCase extends FunSuite with Matchers {
       System.setOut(new PrintStream(baos))
       val args: Array[String] = Array("local")
 
-      //start flink scala shell
-      FlinkShell.bufferedReader = Some(in);
-      FlinkShell.main(args)
+    //start flink scala shell
+    FlinkShell.bufferedReader = Some(in);
+    FlinkShell.main(args)
 
-      baos.flush()
-      val output: String = baos.toString
-      System.setOut(oldOut)
+    baos.flush()
+    val output: String = baos.toString
+    System.setOut(oldOut)
 
-      output should include("Job execution switched to status FINISHED.")
-      output should include("a\nb")
+    Assert.assertTrue(output.contains("IntCounter: 2"))
+    Assert.assertTrue(output.contains("foobar"))
+    Assert.assertTrue(output.contains("barfoo"))
 
-      output should not include "Error"
-      output should not include "ERROR"
-      output should not include "Exception"
-      output should not include "failed"
-    }
+    Assert.assertFalse(output.contains("failed"))
+    Assert.assertFalse(output.contains("Error"))
+    Assert.assertFalse(output.contains("ERROR"))
+    Assert.assertFalse(output.contains("Exception"))
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/630798d3/tools/log4j-travis.properties
----------------------------------------------------------------------
diff --git a/tools/log4j-travis.properties b/tools/log4j-travis.properties
index d55209e..53379b4 100644
--- a/tools/log4j-travis.properties
+++ b/tools/log4j-travis.properties
@@ -40,7 +40,6 @@ log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.apache.zookeeper.server.quorum.QuorumCnxManager=OFF
 log4j.logger.org.apache.flink.runtime.leaderelection=DEBUG
 log4j.logger.org.apache.flink.runtime.leaderretrieval=DEBUG
-log4j.logger.org.apache.flink.runtime.executiongraph=DEBUG
 
 # Log a bit when running the flink-yarn-tests to avoid running into the 5 minutes timeout for
 # the tests


[43/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf
deleted file mode 100644
index ed9372f..0000000
Binary files a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff
deleted file mode 100644
index 8b280b9..0000000
Binary files a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2 b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2
deleted file mode 100644
index 3311d58..0000000
Binary files a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/images/flink-logo.png
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/images/flink-logo.png b/flink-runtime-web/src/main/resources/web/images/flink-logo.png
deleted file mode 100644
index 088fb27..0000000
Binary files a/flink-runtime-web/src/main/resources/web/images/flink-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/index.html b/flink-runtime-web/src/main/resources/web/index.html
deleted file mode 100644
index a6df249..0000000
--- a/flink-runtime-web/src/main/resources/web/index.html
+++ /dev/null
@@ -1,55 +0,0 @@
-
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
---><!DOCTYPE html>
-<html lang="en">
-  <head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <title>Apache Flink Web Dashboard</title>
-    <link rel="stylesheet" href="css/vendor.css" type="text/css">
-    <link rel="stylesheet" href="css/index.css" type="text/css">
-    <script src="js/vendor.js"></script>
-    <script src="js/index.js"></script>
-  </head>
-  <body ng-app="flinkApp" ng-strict-di>
-    <div id="sidebar" ng-class="{ 'sidebar-visible': sidebarVisible }">
-      <nav class="navbar navbar-inverse navbar-static-top">
-        <div class="navbar-header"><a ui-sref="overview" class="navbar-brand"><img alt="Apache Flink Dashboard" src="images/flink-logo.png" class="logo"></a><a ui-sref="overview" class="navbar-brand navbar-brand-text">Apache Flink Dashboard</a></div>
-      </nav>
-      <div class="navbar navbar-sidebar">
-        <ul class="nav">
-          <li><a ui-sref="overview" ui-sref-active="active"><i class="fa fa-dashboard fa-fw"></i> 
-              Overview</a></li>
-          <li><a ui-sref="running-jobs" ui-sref-active="active"><i class="fa fa-tasks fa-fw"></i> 
-              Running Jobs</a></li>
-          <li><a ui-sref="completed-jobs" ui-sref-active="active"><i class="fa fa-check-circle fa-fw"></i> 
-              Completed Jobs</a></li>
-          <li><a ui-sref="all-manager" ui-sref-active="active"><i class="fa fa-sitemap fa-fw"></i> 
-              Task Managers</a></li>
-          <li><a ui-sref="jobmanager.config" ui-sref-active="active"><i class="fa fa-server fa-fw"></i> 
-              Job Manager</a></li>
-        </ul>
-      </div>
-    </div>
-    <div id="content" ng-class="{ 'sidebar-visible': sidebarVisible }">
-      <div ui-view="main"></div>
-    </div>
-  </body>
-</html>
\ No newline at end of file


[27/47] flink git commit: [FLINK-2793] [runtime-web] Rework JobManagerRetriever to avoid race conditions

Posted by tr...@apache.org.
[FLINK-2793] [runtime-web] Rework JobManagerRetriever to avoid race conditions

The JobManagerRetriever sets the new leaderGatewayPortFuture directly in the notifyLeaderAddress
method instead of in one of the futures. This avoids race conditions between multiple futures
which finish in a different order than they were started. Furthermore, this replaces promises
by futures where a promise is not needed.

Add logging statement

Fix WebRuntimeMonitorITCase to use random port and proper state backend

Add ChannelHandler.Sharable to RuntimeMonitorHandler

Remove sanity check from WebInfoServer to let it work on Yarn


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

Branch: refs/heads/master
Commit: e3ad96211ebcab4317a7bb1ba42dfb1a9302aafd
Parents: 77fc0cc
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Oct 9 23:33:25 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:53 2015 +0200

----------------------------------------------------------------------
 flink-runtime-web/pom.xml                       |   8 ++
 .../runtime/webmonitor/JobManagerRetriever.java | 110 ++++++++++---------
 .../webmonitor/RuntimeMonitorHandler.java       |  54 +++++----
 .../runtime/webmonitor/WebRuntimeMonitor.java   |  10 +-
 .../files/StaticFileServerHandler.java          |  63 ++++++-----
 .../handlers/HandlerRedirectUtils.java          |   2 +-
 .../webmonitor/WebRuntimeMonitorITCase.java     |  23 +++-
 .../flink/runtime/jobmanager/JobManager.scala   |  21 ++--
 .../ExecutionGraphRestartTest.scala             |  16 ++-
 tools/log4j-travis.properties                   |   1 +
 10 files changed, 181 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index 727604f..f2ac818 100644
--- a/flink-runtime-web/pom.xml
+++ b/flink-runtime-web/pom.xml
@@ -109,6 +109,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
index 7162639..93db280 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
 import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -35,9 +36,11 @@ import scala.Tuple2;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.Promise;
+import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.util.UUID;
+import java.util.concurrent.TimeoutException;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
@@ -56,17 +59,14 @@ public class JobManagerRetriever implements LeaderRetrievalListener {
 
 	private static final Logger LOG = LoggerFactory.getLogger(JobManagerRetriever.class);
 
-	private final Object lock = new Object();
+	private final Object waitLock = new Object();
 
 	private final WebMonitor webMonitor;
 	private final ActorSystem actorSystem;
 	private final FiniteDuration lookupTimeout;
 	private final FiniteDuration timeout;
 
-	private volatile Tuple2<Promise<ActorGateway>, Promise<Integer>> leaderPromise =
-			new Tuple2<Promise<ActorGateway>, Promise<Integer>>(
-					new scala.concurrent.impl.Promise.DefaultPromise<ActorGateway>(),
-					new scala.concurrent.impl.Promise.DefaultPromise<Integer>());
+	private volatile Future<Tuple2<ActorGateway, Integer>> leaderGatewayPortFuture;
 
 	public JobManagerRetriever(
 			WebMonitor webMonitor,
@@ -81,22 +81,21 @@ public class JobManagerRetriever implements LeaderRetrievalListener {
 	}
 
 	/**
-	 * Returns the leading job manager gateway and its web monitor port.
+	 * Returns the currently known leading job manager gateway and its web monitor port.
 	 */
 	public Option<Tuple2<ActorGateway, Integer>> getJobManagerGatewayAndWebPort() throws Exception {
-		Tuple2<Promise<ActorGateway>, Promise<Integer>> promise = leaderPromise;
+		if (leaderGatewayPortFuture != null) {
+			Future<Tuple2<ActorGateway, Integer>> gatewayPortFuture = leaderGatewayPortFuture;
 
-		if (!promise._1().isCompleted() || !promise._1().isCompleted()) {
-			return Option.empty();
-		}
-		else {
-			Promise<ActorGateway> leaderGatewayPromise = promise._1();
-			Promise<Integer> leaderWebPortPromise = promise._2();
+			if (gatewayPortFuture.isCompleted()) {
+				Tuple2<ActorGateway, Integer> gatewayPort = Await.result(gatewayPortFuture, timeout);
 
-			ActorGateway leaderGateway = Await.result(leaderGatewayPromise.future(), timeout);
-			int leaderWebPort = Await.result(leaderWebPortPromise.future(), timeout);
-
-			return Option.apply(new Tuple2<>(leaderGateway, leaderWebPort));
+				return Option.apply(gatewayPort);
+			} else {
+				return Option.empty();
+			}
+		} else {
+			return Option.empty();
 		}
 	}
 
@@ -104,66 +103,73 @@ public class JobManagerRetriever implements LeaderRetrievalListener {
 	 * Awaits the leading job manager gateway and its web monitor port.
 	 */
 	public Tuple2<ActorGateway, Integer> awaitJobManagerGatewayAndWebPort() throws Exception {
-		Tuple2<Promise<ActorGateway>, Promise<Integer>> promise = leaderPromise;
+		Future<Tuple2<ActorGateway, Integer>> gatewayPortFuture = null;
+		Deadline deadline = timeout.fromNow();
 
-		Promise<ActorGateway> leaderGatewayPromise = promise._1();
-		Promise<Integer> leaderWebPortPromise = promise._2();
+		while(!deadline.isOverdue()) {
+			synchronized (waitLock) {
+				gatewayPortFuture = leaderGatewayPortFuture;
 
-		ActorGateway leaderGateway = Await.result(leaderGatewayPromise.future(), timeout);
-		int leaderWebPort = Await.result(leaderWebPortPromise.future(), timeout);
+				if (gatewayPortFuture != null) {
+					break;
+				}
 
-		return new Tuple2<>(leaderGateway, leaderWebPort);
+				waitLock.wait(deadline.timeLeft().toMillis());
+			}
+		}
+
+		if (gatewayPortFuture == null) {
+			throw new TimeoutException("There is no JobManager available.");
+		} else {
+			return Await.result(gatewayPortFuture, deadline.timeLeft());
+		}
 	}
 
 	@Override
 	public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
 		if (leaderAddress != null && !leaderAddress.equals("")) {
 			try {
-				final Promise<ActorGateway> gatewayPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
-				final Promise<Integer> webPortPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+				final Promise<Tuple2<ActorGateway, Integer>> leaderGatewayPortPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
 
-				final Tuple2<Promise<ActorGateway>, Promise<Integer>> newPromise = new Tuple2<>(
-						gatewayPromise, webPortPromise);
+				synchronized (waitLock) {
+					leaderGatewayPortFuture = leaderGatewayPortPromise.future();
+					waitLock.notifyAll();
+				}
 
-				LOG.info("Retrieved leader notification {}:{}.", leaderAddress, leaderSessionID);
+				LOG.info("New leader reachable under {}:{}.", leaderAddress, leaderSessionID);
 
 				AkkaUtils.getActorRefFuture(leaderAddress, actorSystem, lookupTimeout)
 						// Resolve the actor ref
-						.flatMap(new Mapper<ActorRef, Future<Object>>() {
+						.flatMap(new Mapper<ActorRef, Future<Tuple2<ActorGateway, Object>>>() {
 							@Override
-							public Future<Object> apply(ActorRef jobManagerRef) {
+							public Future<Tuple2<ActorGateway, Object>> apply(ActorRef jobManagerRef) {
 								ActorGateway leaderGateway = new AkkaActorGateway(
 										jobManagerRef, leaderSessionID);
 
-								gatewayPromise.success(leaderGateway);
+								Future<Object> webMonitorPort = leaderGateway.ask(
+									JobManagerMessages.getRequestWebMonitorPort(),
+									timeout);
 
-								return leaderGateway.ask(JobManagerMessages
-										.getRequestWebMonitorPort(), timeout);
+								return Futures.successful(leaderGateway).zip(webMonitorPort);
 							}
 						}, actorSystem.dispatcher())
 								// Request the web monitor port
-						.onComplete(new OnComplete<Object>() {
+						.onComplete(new OnComplete<Tuple2<ActorGateway, Object>>() {
 							@Override
-							public void onComplete(Throwable failure, Object success) throws Throwable {
+							public void onComplete(Throwable failure, Tuple2<ActorGateway, Object> success) throws Throwable {
 								if (failure == null) {
-									int webMonitorPort = ((ResponseWebMonitorPort) success).port();
-									webPortPromise.success(webMonitorPort);
-
-									// Complete the promise
-									synchronized (lock) {
-										Tuple2<Promise<ActorGateway>, Promise<Integer>>
-												previousPromise = leaderPromise;
-
-										leaderPromise = newPromise;
-
-										if (!previousPromise._2().isCompleted()) {
-											previousPromise._1().completeWith(gatewayPromise.future());
-											previousPromise._2().completeWith(webPortPromise.future());
-										}
+									if (success._2() instanceof ResponseWebMonitorPort) {
+										int webMonitorPort = ((ResponseWebMonitorPort) success._2()).port();
+
+										leaderGatewayPortPromise.success(new Tuple2<>(success._1(), webMonitorPort));
+									} else {
+										leaderGatewayPortPromise.failure(new Exception("Received the message " +
+										success._2() + " as response to " + JobManagerMessages.getRequestWebMonitorPort() +
+											". But a message of type " + ResponseWebMonitorPort.class + " was expected."));
 									}
-								}
-								else {
-									LOG.warn("Failed to retrieve leader gateway and port.");
+								} else {
+									LOG.warn("Failed to retrieve leader gateway and port.", failure);
+									leaderGatewayPortPromise.failure(failure);
 								}
 							}
 						}, actorSystem.dispatcher());

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index b9369ea..e174463 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.handler.codec.http.DefaultFullHttpResponse;
@@ -35,7 +36,9 @@ import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
 import org.apache.flink.util.ExceptionUtils;
 import scala.Option;
 import scala.Tuple2;
-import scala.concurrent.Promise;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.nio.charset.Charset;
 
@@ -47,6 +50,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * This handler also deals with setting correct response MIME types and returning
  * proper codes, like OK, NOT_FOUND, or SERVER_ERROR.
  */
+@ChannelHandler.Sharable
 public class RuntimeMonitorHandler extends SimpleChannelInboundHandler<Routed> {
 
 	private static final Charset ENCODING = Charset.forName("UTF-8");
@@ -55,7 +59,9 @@ public class RuntimeMonitorHandler extends SimpleChannelInboundHandler<Routed> {
 
 	private final JobManagerRetriever retriever;
 
-	private final Promise<String> localJobManagerAddressPromise;
+	private final Future<String> localJobManagerAddressFuture;
+
+	private final FiniteDuration timeout;
 
 	private final String contentType;
 
@@ -64,35 +70,41 @@ public class RuntimeMonitorHandler extends SimpleChannelInboundHandler<Routed> {
 	public RuntimeMonitorHandler(
 			RequestHandler handler,
 			JobManagerRetriever retriever,
-			Promise<String> localJobManagerAddressPromise) {
+			Future<String> localJobManagerAddressFuture,
+			FiniteDuration timeout) {
 
 		this.handler = checkNotNull(handler);
 		this.retriever = checkNotNull(retriever);
-		this.localJobManagerAddressPromise = checkNotNull(localJobManagerAddressPromise);
+		this.localJobManagerAddressFuture = checkNotNull(localJobManagerAddressFuture);
+		this.timeout = checkNotNull(timeout);
 		this.contentType = (handler instanceof RequestHandler.JsonResponse) ? "application/json" : "text/plain";
 	}
 
 	@Override
 	protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
-		if (localJobManagerAddress == null) {
-			localJobManagerAddress = localJobManagerAddressPromise.future().value().get().get();
-		}
-
-		Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
-
-		if (jobManager.isDefined()) {
-			String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
-					localJobManagerAddress, jobManager.get());
-
-			if (redirectAddress != null) {
-				HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, routed.path());
-				KeepAliveWrite.flush(ctx, routed.request(), redirect);
+		if (localJobManagerAddressFuture.isCompleted()) {
+			if (localJobManagerAddress == null) {
+				localJobManagerAddress = Await.result(localJobManagerAddressFuture, timeout);
 			}
-			else {
-				respondAsLeader(ctx, routed, jobManager.get()._1());
+
+			Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
+
+			if (jobManager.isDefined()) {
+				Tuple2<ActorGateway, Integer> gatewayPort = jobManager.get();
+				String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
+					localJobManagerAddress, gatewayPort);
+
+				if (redirectAddress != null) {
+					HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, routed.path());
+					KeepAliveWrite.flush(ctx, routed.request(), redirect);
+				}
+				else {
+					respondAsLeader(ctx, routed, gatewayPort._1());
+				}
+			} else {
+				KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
 			}
-		}
-		else {
+		} else {
 			KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 40ab6c1..ec973c7 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -115,6 +115,8 @@ public class WebRuntimeMonitor implements WebMonitor {
 
 	private final Promise<String> jobManagerAddressPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
 
+	private final FiniteDuration timeout;
+
 	private Channel serverChannel;
 
 	private final File webRootDir;
@@ -174,7 +176,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			throw new IllegalArgumentException("Web frontend port is invalid: " + this.configuredPort);
 		}
 
-		FiniteDuration timeout = AkkaUtils.getTimeout(config);
+		timeout = AkkaUtils.getTimeout(config);
 		FiniteDuration lookupTimeout = AkkaUtils.getTimeout(config);
 
 		retriever = new JobManagerRetriever(this, actorSystem, lookupTimeout, timeout);
@@ -218,10 +220,10 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/jobs/:jobid/accumulators", handler(new JobAccumulatorsHandler(currentGraphs)))
 
 			.GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT)))
-			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
+			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT)))
 
 			// this handler serves all the static contents
-			.GET("/:*", new StaticFileServerHandler(retriever, jobManagerAddressPromise, webRootDir));
+			.GET("/:*", new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, webRootDir));
 
 		synchronized (startupShutdownLock) {
 
@@ -335,6 +337,6 @@ public class WebRuntimeMonitor implements WebMonitor {
 	// ------------------------------------------------------------------------
 
 	private RuntimeMonitorHandler handler(RequestHandler handler) {
-		return new RuntimeMonitorHandler(handler, retriever, jobManagerAddressPromise);
+		return new RuntimeMonitorHandler(handler, retriever, jobManagerAddressPromise.future(), timeout);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
index 944407e..d46a900 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
@@ -52,7 +52,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.Option;
 import scala.Tuple2;
-import scala.concurrent.Promise;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -108,7 +110,9 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 	/** JobManager retriever */
 	private final JobManagerRetriever retriever;
 
-	private final Promise<String> localJobManagerAddressPromise;
+	private final Future<String> localJobManagerAddressFuture;
+
+	private final FiniteDuration timeout;
 
 	/** The path in which the static documents are */
 	private final File rootPath;
@@ -120,20 +124,23 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 
 	public StaticFileServerHandler(
 			JobManagerRetriever retriever,
-			Promise<String> localJobManagerAddressPromise,
+			Future<String> localJobManagerAddressPromise,
+			FiniteDuration timeout,
 			File rootPath) {
 
-		this(retriever, localJobManagerAddressPromise, rootPath, DEFAULT_LOGGER);
+		this(retriever, localJobManagerAddressPromise, timeout, rootPath, DEFAULT_LOGGER);
 	}
 
 	public StaticFileServerHandler(
 			JobManagerRetriever retriever,
-			Promise<String> localJobManagerAddressPromise,
+			Future<String> localJobManagerAddressFuture,
+			FiniteDuration timeout,
 			File rootPath,
 			Logger logger) {
 
 		this.retriever = checkNotNull(retriever);
-		this.localJobManagerAddressPromise = localJobManagerAddressPromise;
+		this.localJobManagerAddressFuture = checkNotNull(localJobManagerAddressFuture);
+		this.timeout = checkNotNull(timeout);
 		this.rootPath = checkNotNull(rootPath);
 		this.logger = checkNotNull(logger);
 	}
@@ -144,41 +151,45 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 
 	@Override
 	public void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
-		if (localJobManagerAddress == null) {
-			localJobManagerAddress = localJobManagerAddressPromise.future().value().get().get();
-		}
+		if (localJobManagerAddressFuture.isCompleted()) {
+			if (localJobManagerAddress == null) {
+				localJobManagerAddress = Await.result(localJobManagerAddressFuture, timeout);
+			}
 
-		final HttpRequest request = routed.request();
-		String requestPath = routed.path();
+			final HttpRequest request = routed.request();
+			String requestPath = routed.path();
 
-		// make sure we request the "index.html" in case there is a directory request
-		if (requestPath.endsWith("/")) {
-			requestPath = requestPath + "index.html";
-		}
+			// make sure we request the "index.html" in case there is a directory request
+			if (requestPath.endsWith("/")) {
+				requestPath = requestPath + "index.html";
+			}
 
 		// in case the files being accessed are logs or stdout files, find appropriate paths.
 		if (requestPath.equals("/jobmanager/log")) {
 			requestPath = "/" + getFileName(rootPath, WebRuntimeMonitor.LOG_FILE_PATTERN);
 		} else if (requestPath.equals("/jobmanager/stdout")) {
 			requestPath = "/" + getFileName(rootPath, WebRuntimeMonitor.STDOUT_FILE_PATTERN);
-		}
+			}
 
-		Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
+			Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
 
-		if (jobManager.isDefined()) {
-			// Redirect to leader if necessary
-			String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
+			if (jobManager.isDefined()) {
+				// Redirect to leader if necessary
+				String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
 					localJobManagerAddress, jobManager.get());
 
-			if (redirectAddress != null) {
-				HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, requestPath);
-				KeepAliveWrite.flush(ctx, routed.request(), redirect);
+				if (redirectAddress != null) {
+					HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, requestPath);
+					KeepAliveWrite.flush(ctx, routed.request(), redirect);
+				}
+				else {
+					respondAsLeader(ctx, request, requestPath);
+				}
 			}
 			else {
-				respondAsLeader(ctx, request, requestPath);
+				KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
 			}
-		}
-		else {
+		} else {
 			KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
index 887c46e..800c7c0 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
@@ -67,7 +67,7 @@ public class HandlerRedirectUtils {
 				return redirectAddress;
 			}
 			else {
-				LOG.warn("Unexpected leader address pattern. Cannot extract host.");
+				LOG.warn("Unexpected leader address pattern {}. Cannot extract host.", leaderAddress);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 26f66b0..5167d13 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -27,13 +27,18 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
+import org.apache.flink.util.TestLogger;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.powermock.reflect.Whitebox;
 import scala.Some;
 import scala.Tuple2;
@@ -50,7 +55,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
-public class WebRuntimeMonitorITCase {
+public class WebRuntimeMonitorITCase extends TestLogger {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
 	private final static FiniteDuration TestTimeout = new FiniteDuration(2, TimeUnit.MINUTES);
 
@@ -77,6 +85,7 @@ public class WebRuntimeMonitorITCase {
 			Configuration monitorConfig = new Configuration();
 			monitorConfig.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
 			monitorConfig.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
+			monitorConfig.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
 
 			// Needs to match the leader address from the leader retrieval service
 			String jobManagerAddress = AkkaUtils.getAkkaURL(jmActorSystem, jmActor);
@@ -131,12 +140,12 @@ public class WebRuntimeMonitorITCase {
 		List<LeaderRetrievalService> leaderRetrievalServices = new ArrayList<>();
 
 		try (TestingServer zooKeeper = new TestingServer()) {
-			final Configuration config = new Configuration();
+			final Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+				zooKeeper.getConnectString(),
+				temporaryFolder.getRoot().getPath());
 			config.setString(WebMonitorConfig.JOB_MANAGER_WEB_DOC_ROOT_KEY, MAIN_RESOURCES_PATH);
 			config.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
 			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
-			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
-			config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
 
 			for (int i = 0; i < jobManagerSystem.length; i++) {
 				jobManagerSystem[i] = AkkaUtils.createActorSystem(new Configuration(),
@@ -157,7 +166,11 @@ public class WebRuntimeMonitorITCase {
 						webMonitor[i].getServerPort());
 
 				jobManager[i] = JobManager.startJobManagerActors(
-						jmConfig, jobManagerSystem[i], StreamingMode.STREAMING)._1();
+					jmConfig,
+					jobManagerSystem[i],
+					StreamingMode.STREAMING,
+					JobManager.class,
+					MemoryArchivist.class)._1();
 
 				jobManagerAddress[i] = AkkaUtils.getAkkaURL(jobManagerSystem[i], jobManager[i]);
 				webMonitor[i].start(jobManagerAddress[i]);

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/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 161e8de..ebc0ea9 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
@@ -1624,7 +1624,6 @@ object JobManager {
         monitor =>
           val jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(configuration)
           monitor.start(jobManagerAkkaUrl)
-
         LOG.info("Starting JobManger web frontend")
         // start the web frontend. we need to load this dynamically
         // because it is not in the same project/dependencies
@@ -1632,11 +1631,9 @@ object JobManager {
           configuration,
           leaderRetrievalService,
           jobManagerSystem)
-        Option(webServer)
-      } else {
-        None
       }
 
+
       (jobManagerSystem, jobManager, archive, webMonitor)
     }
     catch {
@@ -1930,14 +1927,14 @@ object JobManager {
    * @return A tuple of references (JobManager Ref, Archiver Ref)
    */
   def startJobManagerActors(
-                             configuration: Configuration,
-                             actorSystem: ActorSystem,
-                             jobMangerActorName: Option[String],
-                             archiveActorName: Option[String],
-                             streamingMode: StreamingMode,
-                             jobManagerClass: Class[_ <: JobManager],
-                             archiveClass: Class[_ <: MemoryArchivist])
-  : (ActorRef, ActorRef) = {
+      configuration: Configuration,
+      actorSystem: ActorSystem,
+      jobMangerActorName: Option[String],
+      archiveActorName: Option[String],
+      streamingMode: StreamingMode,
+      jobManagerClass: Class[_ <: JobManager],
+      archiveClass: Class[_ <: MemoryArchivist])
+    : (ActorRef, ActorRef) = {
 
     val (executionContext,
     instanceManager,

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
index 9a1cde0..e41d7ff 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
@@ -144,15 +144,19 @@ class ExecutionGraphRestartTest extends WordSpecLike with Matchers {
         // Wait for deploying after async restart
         deadline = timeout.fromNow
         while (deadline.hasTimeLeft() && eg.getAllExecutionVertices.asScala.exists(
-          _.getCurrentExecutionAttempt.getState != ExecutionState.DEPLOYING)) {
+          _.getCurrentExecutionAttempt.getAssignedResource == null)) {
           Thread.sleep(100)
         }
-        
-        for (vertex <- eg.getAllExecutionVertices.asScala) {
-          vertex.getCurrentExecutionAttempt().markFinished()
-        }
 
-        eg.getState() should equal(JobStatus.FINISHED)
+        if (deadline.hasTimeLeft()) {
+          for (vertex <- eg.getAllExecutionVertices.asScala) {
+            vertex.getCurrentExecutionAttempt().markFinished()
+          }
+
+          eg.getState() should equal(JobStatus.FINISHED)
+        } else {
+          fail("Failed to wait until all execution attempts left the state DEPLOYING.")
+        }
       } catch {
         case t: Throwable =>
           t.printStackTrace()

http://git-wip-us.apache.org/repos/asf/flink/blob/e3ad9621/tools/log4j-travis.properties
----------------------------------------------------------------------
diff --git a/tools/log4j-travis.properties b/tools/log4j-travis.properties
index 53379b4..d55209e 100644
--- a/tools/log4j-travis.properties
+++ b/tools/log4j-travis.properties
@@ -40,6 +40,7 @@ log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.apache.zookeeper.server.quorum.QuorumCnxManager=OFF
 log4j.logger.org.apache.flink.runtime.leaderelection=DEBUG
 log4j.logger.org.apache.flink.runtime.leaderretrieval=DEBUG
+log4j.logger.org.apache.flink.runtime.executiongraph=DEBUG
 
 # Log a bit when running the flink-yarn-tests to avoid running into the 5 minutes timeout for
 # the tests


[25/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
[FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Squashes:
5a88d5e [tests] Add HttpTestClient for testing HTTP responses
656d6d6 Split WebMonitor and LeaderRetrievalService start up
a7e8da8 Move generated /web files to src/main/resources

Add comment to webMonitorPort attribute and make line breaks more Scalaesque

Don't block on leader retrieval and only resolve associated job manager once

Make JobManagerRetriever independent of redirecting logic

This closes #1202.


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

Branch: refs/heads/master
Commit: 77fc0cc445e14eeef1952e0760f80912351574bd
Parents: d18f580
Author: Ufuk Celebi <uc...@apache.org>
Authored: Fri Sep 25 11:44:53 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:52 2015 +0200

----------------------------------------------------------------------
 flink-runtime-web/README.md                     |     2 +-
 flink-runtime-web/pom.xml                       |    18 +
 .../webmonitor/ExecutionGraphHolder.java        |    21 +-
 .../webmonitor/JobManagerArchiveRetriever.java  |   111 -
 .../runtime/webmonitor/JobManagerRetriever.java |   189 +
 .../webmonitor/RuntimeMonitorHandler.java       |    69 +-
 .../runtime/webmonitor/WebRuntimeMonitor.java   |    60 +-
 .../files/StaticFileServerHandler.java          |   102 +-
 .../AbstractExecutionGraphRequestHandler.java   |     9 +-
 .../handlers/ClusterOverviewHandler.java        |    20 +-
 .../handlers/CurrentJobIdsHandler.java          |    10 +-
 .../handlers/CurrentJobsOverviewHandler.java    |    24 +-
 .../handlers/DashboardConfigHandler.java        |     3 +-
 .../handlers/HandlerRedirectUtils.java          |   105 +
 .../handlers/JobManagerConfigHandler.java       |     3 +-
 .../webmonitor/handlers/RequestHandler.java     |     8 +-
 .../handlers/TaskManagersHandler.java           |    22 +-
 .../src/main/resources/web/css/index.css        |   596 +
 .../src/main/resources/web/css/vendor.css       |  9183 ++
 .../main/resources/web/fonts/FontAwesome.otf    |   Bin 0 -> 93888 bytes
 .../resources/web/fonts/fontawesome-webfont.eot |   Bin 0 -> 60767 bytes
 .../resources/web/fonts/fontawesome-webfont.svg |   565 +
 .../resources/web/fonts/fontawesome-webfont.ttf |   Bin 0 -> 122092 bytes
 .../web/fonts/fontawesome-webfont.woff          |   Bin 0 -> 71508 bytes
 .../web/fonts/fontawesome-webfont.woff2         |   Bin 0 -> 56780 bytes
 .../main/resources/web/images/flink-logo.png    |   Bin 0 -> 6096 bytes
 .../src/main/resources/web/index.html           |    55 +
 .../src/main/resources/web/js/index.js          |  1420 +
 .../src/main/resources/web/js/index.js.orig     |  1150 +
 .../src/main/resources/web/js/vendor.js         | 81087 +++++++++++++++++
 .../web/partials/jobmanager/config.html         |    33 +
 .../web/partials/jobmanager/index.html          |    33 +
 .../web/partials/jobmanager/stdout.html         |    40 +
 .../web/partials/jobs/completed-jobs.html       |    53 +
 .../resources/web/partials/jobs/job.config.html |    57 +
 .../web/partials/jobs/job.exceptions.html       |    38 +
 .../main/resources/web/partials/jobs/job.html   |    48 +
 .../resources/web/partials/jobs/job.plan.html   |    31 +
 .../jobs/job.plan.node-list.accumulators.html   |    40 +
 .../jobs/job.plan.node-list.overview.html       |    60 +
 .../jobs/job.plan.node.accumulators.html        |    68 +
 .../partials/jobs/job.plan.node.subtasks.html   |    52 +
 .../web/partials/jobs/job.properties.html       |   140 +
 .../web/partials/jobs/job.statistics.html       |    40 +
 .../web/partials/jobs/job.timeline.html         |    23 +
 .../web/partials/jobs/job.timeline.vertex.html  |    30 +
 .../web/partials/jobs/running-jobs.html         |    53 +
 .../main/resources/web/partials/overview.html   |   147 +
 .../webmonitor/WebRuntimeMonitorITCase.java     |   329 +
 .../webmonitor/testutils/HttpTestClient.java    |   309 +
 flink-runtime-web/web-dashboard/gulpfile.js     |     4 +-
 .../web-dashboard/web/css/index.css             |   596 -
 .../web-dashboard/web/css/vendor.css            |  9183 --
 .../web-dashboard/web/fonts/FontAwesome.otf     |   Bin 93888 -> 0 bytes
 .../web/fonts/fontawesome-webfont.eot           |   Bin 60767 -> 0 bytes
 .../web/fonts/fontawesome-webfont.svg           |   565 -
 .../web/fonts/fontawesome-webfont.ttf           |   Bin 122092 -> 0 bytes
 .../web/fonts/fontawesome-webfont.woff          |   Bin 71508 -> 0 bytes
 .../web/fonts/fontawesome-webfont.woff2         |   Bin 56780 -> 0 bytes
 .../web-dashboard/web/images/flink-logo.png     |   Bin 6096 -> 0 bytes
 flink-runtime-web/web-dashboard/web/index.html  |    55 -
 flink-runtime-web/web-dashboard/web/js/index.js |  1420 -
 .../web-dashboard/web/js/vendor.js              | 81087 -----------------
 .../web/partials/jobmanager/config.html         |    33 -
 .../web/partials/jobmanager/index.html          |    33 -
 .../web/partials/jobmanager/stdout.html         |    40 -
 .../web/partials/jobs/completed-jobs.html       |    53 -
 .../web/partials/jobs/job.config.html           |    57 -
 .../web/partials/jobs/job.exceptions.html       |    38 -
 .../web-dashboard/web/partials/jobs/job.html    |    48 -
 .../web/partials/jobs/job.plan.html             |    31 -
 .../jobs/job.plan.node-list.accumulators.html   |    40 -
 .../jobs/job.plan.node-list.overview.html       |    60 -
 .../jobs/job.plan.node.accumulators.html        |    68 -
 .../partials/jobs/job.plan.node.subtasks.html   |    52 -
 .../web/partials/jobs/job.properties.html       |   140 -
 .../web/partials/jobs/job.statistics.html       |    40 -
 .../web/partials/jobs/job.timeline.html         |    23 -
 .../web/partials/jobs/job.timeline.vertex.html  |    30 -
 .../web/partials/jobs/running-jobs.html         |    53 -
 .../web-dashboard/web/partials/overview.html    |   147 -
 .../web/partials/taskmanager/index.html         |    57 -
 .../runtime/instance/AkkaActorGateway.java      |     5 +
 .../flink/runtime/util/StandaloneUtils.java     |     2 +
 .../flink/runtime/webmonitor/WebMonitor.java    |     2 +-
 .../flink/runtime/jobmanager/JobManager.scala   |    97 +-
 .../runtime/messages/JobManagerMessages.scala   |    14 +
 .../runtime/minicluster/FlinkMiniCluster.scala  |     2 +-
 .../apache/flink/yarn/ApplicationMaster.scala   |     1 +
 pom.xml                                         |     2 +-
 90 files changed, 96305 insertions(+), 94229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/README.md
----------------------------------------------------------------------
diff --git a/flink-runtime-web/README.md b/flink-runtime-web/README.md
index 9664681..b724163 100644
--- a/flink-runtime-web/README.md
+++ b/flink-runtime-web/README.md
@@ -93,7 +93,7 @@ bower install
 gulp
 ```
 
-The dashboard code is under `/app`. The result of the build process is under `/web`.
+The dashboard code is under `/app`. The result of the build process is under `/web` in the src/main/resources folder.
 
 When building Flink with Maven (in particular the `flink-dist` project), the generated
 files are copied into the build target, to the folder `resources/web-runtime-monitor`.

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index ffb68bc..727604f 100644
--- a/flink-runtime-web/pom.xml
+++ b/flink-runtime-web/pom.xml
@@ -103,6 +103,24 @@ under the License.
 			<version>${guava.version}</version>
 		</dependency>
 
+		<!-- ===================================================
+								Testing
+			=================================================== -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
 	</dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
index 09ede4c..f680306 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
@@ -31,6 +31,8 @@ import scala.concurrent.duration.FiniteDuration;
 
 import java.util.WeakHashMap;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
  * <p>
@@ -43,23 +45,16 @@ public class ExecutionGraphHolder {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraphHolder.class);
 
-	/** Retrieves the current leading JobManager and its corresponding archive */
-	private final JobManagerArchiveRetriever retriever;
-
 	private final FiniteDuration timeout;
 
 	private final WeakHashMap<JobID, ExecutionGraph> cache = new WeakHashMap<JobID, ExecutionGraph>();
 
-	public ExecutionGraphHolder(JobManagerArchiveRetriever retriever) {
-		this(retriever, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
+	public ExecutionGraphHolder() {
+		this(WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
 	}
 
-	public ExecutionGraphHolder(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
-		if (retriever == null || timeout == null) {
-			throw new NullPointerException();
-		}
-		this.retriever = retriever;
-		this.timeout = timeout;
+	public ExecutionGraphHolder(FiniteDuration timeout) {
+		this.timeout = checkNotNull(timeout);
 	}
 
 	/**
@@ -68,15 +63,13 @@ public class ExecutionGraphHolder {
 	 * @param jid jobID of the execution graph to be retrieved
 	 * @return the retrieved execution graph or null if it is not retrievable
 	 */
-	public ExecutionGraph getExecutionGraph(JobID jid) {
+	public ExecutionGraph getExecutionGraph(JobID jid, ActorGateway jobManager) {
 		ExecutionGraph cached = cache.get(jid);
 		if (cached != null) {
 			return cached;
 		}
 
 		try {
-			ActorGateway jobManager = retriever.getJobManagerGateway();
-
 			if (jobManager != null) {
 				Future<Object> future = jobManager.ask(new JobManagerMessages.RequestJob(jid), timeout);
 				Object result = Await.result(future, timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
deleted file mode 100644
index 91c9ad5..0000000
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.webmonitor;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-
-/**
- * Retrieves and stores the actor gateway to the current leading JobManager and its archive. In
- * case of an error, the {@link WebRuntimeMonitor} to which this instance is associated will be
- * stopped.
- */
-public class JobManagerArchiveRetriever implements LeaderRetrievalListener {
-
-	private static final Logger LOG = LoggerFactory.getLogger(JobManagerArchiveRetriever.class);
-
-	private final ActorSystem actorSystem;
-	private final FiniteDuration lookupTimeout;
-	private final FiniteDuration timeout;
-	private final WebMonitor webMonitor;
-
-	/** will be written and read concurrently */
-	private volatile ActorGateway jobManagerGateway;
-	private volatile ActorGateway archiveGateway;
-
-	public JobManagerArchiveRetriever(
-			WebMonitor webMonitor,
-			ActorSystem actorSystem,
-			FiniteDuration lookupTimeout,
-			FiniteDuration timeout) {
-		this.webMonitor = webMonitor;
-		this.actorSystem = actorSystem;
-		this.lookupTimeout = lookupTimeout;
-		this.timeout = timeout;
-	}
-
-	public ActorGateway getJobManagerGateway() {
-		return jobManagerGateway;
-	}
-
-	public ActorGateway getArchiveGateway() {
-		return archiveGateway;
-	}
-
-
-	@Override
-	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
-		if (leaderAddress != null && !leaderAddress.equals("")) {
-			try {
-				ActorRef jobManager = AkkaUtils.getActorRef(
-						leaderAddress,
-						actorSystem,
-						lookupTimeout);
-				jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID);
-
-				Future<Object> archiveFuture = jobManagerGateway.ask(
-						JobManagerMessages.getRequestArchive(),
-						timeout);
-
-				ActorRef archive = ((JobManagerMessages.ResponseArchive) Await.result(
-						archiveFuture,
-						timeout)
-				).actor();
-
-				archiveGateway = new AkkaActorGateway(archive, leaderSessionID);
-			} catch (Exception e) {
-				handleError(e);
-			}
-		}
-	}
-
-	@Override
-	public void handleError(Exception exception) {
-		LOG.error("Received error from LeaderRetrievalService.", exception);
-
-		try{
-			// stop associated webMonitor
-			webMonitor.stop();
-		} catch (Exception e) {
-			LOG.error("Error while stopping the web server due to a LeaderRetrievalService error.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
new file mode 100644
index 0000000..7162639
--- /dev/null
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerRetriever.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.webmonitor;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.dispatch.Mapper;
+import akka.dispatch.OnComplete;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.JobManagerMessages.ResponseWebMonitorPort;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Retrieves and stores the actor gateway to the current leading JobManager. In case of an error,
+ * the {@link WebRuntimeMonitor} to which this instance is associated will be stopped.
+ *
+ * <p>The job manager gateway only works if the web monitor and the job manager run in the same
+ * actor system, because many execution graph structures are not serializable. This breaks the nice
+ * leader retrieval abstraction and we have a special code path in case that another job manager is
+ * leader (see {@link org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils}. In such a
+ * case, we get the address of the web monitor of the leading job manager and redirect to it
+ * (instead of directly communicating with it).
+ */
+public class JobManagerRetriever implements LeaderRetrievalListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JobManagerRetriever.class);
+
+	private final Object lock = new Object();
+
+	private final WebMonitor webMonitor;
+	private final ActorSystem actorSystem;
+	private final FiniteDuration lookupTimeout;
+	private final FiniteDuration timeout;
+
+	private volatile Tuple2<Promise<ActorGateway>, Promise<Integer>> leaderPromise =
+			new Tuple2<Promise<ActorGateway>, Promise<Integer>>(
+					new scala.concurrent.impl.Promise.DefaultPromise<ActorGateway>(),
+					new scala.concurrent.impl.Promise.DefaultPromise<Integer>());
+
+	public JobManagerRetriever(
+			WebMonitor webMonitor,
+			ActorSystem actorSystem,
+			FiniteDuration lookupTimeout,
+			FiniteDuration timeout) {
+
+		this.webMonitor = checkNotNull(webMonitor);
+		this.actorSystem = checkNotNull(actorSystem);
+		this.lookupTimeout = checkNotNull(lookupTimeout);
+		this.timeout = checkNotNull(timeout);
+	}
+
+	/**
+	 * Returns the leading job manager gateway and its web monitor port.
+	 */
+	public Option<Tuple2<ActorGateway, Integer>> getJobManagerGatewayAndWebPort() throws Exception {
+		Tuple2<Promise<ActorGateway>, Promise<Integer>> promise = leaderPromise;
+
+		if (!promise._1().isCompleted() || !promise._1().isCompleted()) {
+			return Option.empty();
+		}
+		else {
+			Promise<ActorGateway> leaderGatewayPromise = promise._1();
+			Promise<Integer> leaderWebPortPromise = promise._2();
+
+			ActorGateway leaderGateway = Await.result(leaderGatewayPromise.future(), timeout);
+			int leaderWebPort = Await.result(leaderWebPortPromise.future(), timeout);
+
+			return Option.apply(new Tuple2<>(leaderGateway, leaderWebPort));
+		}
+	}
+
+	/**
+	 * Awaits the leading job manager gateway and its web monitor port.
+	 */
+	public Tuple2<ActorGateway, Integer> awaitJobManagerGatewayAndWebPort() throws Exception {
+		Tuple2<Promise<ActorGateway>, Promise<Integer>> promise = leaderPromise;
+
+		Promise<ActorGateway> leaderGatewayPromise = promise._1();
+		Promise<Integer> leaderWebPortPromise = promise._2();
+
+		ActorGateway leaderGateway = Await.result(leaderGatewayPromise.future(), timeout);
+		int leaderWebPort = Await.result(leaderWebPortPromise.future(), timeout);
+
+		return new Tuple2<>(leaderGateway, leaderWebPort);
+	}
+
+	@Override
+	public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
+		if (leaderAddress != null && !leaderAddress.equals("")) {
+			try {
+				final Promise<ActorGateway> gatewayPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+				final Promise<Integer> webPortPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+
+				final Tuple2<Promise<ActorGateway>, Promise<Integer>> newPromise = new Tuple2<>(
+						gatewayPromise, webPortPromise);
+
+				LOG.info("Retrieved leader notification {}:{}.", leaderAddress, leaderSessionID);
+
+				AkkaUtils.getActorRefFuture(leaderAddress, actorSystem, lookupTimeout)
+						// Resolve the actor ref
+						.flatMap(new Mapper<ActorRef, Future<Object>>() {
+							@Override
+							public Future<Object> apply(ActorRef jobManagerRef) {
+								ActorGateway leaderGateway = new AkkaActorGateway(
+										jobManagerRef, leaderSessionID);
+
+								gatewayPromise.success(leaderGateway);
+
+								return leaderGateway.ask(JobManagerMessages
+										.getRequestWebMonitorPort(), timeout);
+							}
+						}, actorSystem.dispatcher())
+								// Request the web monitor port
+						.onComplete(new OnComplete<Object>() {
+							@Override
+							public void onComplete(Throwable failure, Object success) throws Throwable {
+								if (failure == null) {
+									int webMonitorPort = ((ResponseWebMonitorPort) success).port();
+									webPortPromise.success(webMonitorPort);
+
+									// Complete the promise
+									synchronized (lock) {
+										Tuple2<Promise<ActorGateway>, Promise<Integer>>
+												previousPromise = leaderPromise;
+
+										leaderPromise = newPromise;
+
+										if (!previousPromise._2().isCompleted()) {
+											previousPromise._1().completeWith(gatewayPromise.future());
+											previousPromise._2().completeWith(webPortPromise.future());
+										}
+									}
+								}
+								else {
+									LOG.warn("Failed to retrieve leader gateway and port.");
+								}
+							}
+						}, actorSystem.dispatcher());
+			}
+			catch (Exception e) {
+				handleError(e);
+			}
+		}
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		LOG.error("Received error from LeaderRetrievalService.", exception);
+
+		try {
+			// stop associated webMonitor
+			webMonitor.stop();
+		}
+		catch (Exception e) {
+			LOG.error("Error while stopping the web server due to a LeaderRetrievalService error.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index 4574519..b9369ea 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -20,51 +20,90 @@ package org.apache.flink.runtime.webmonitor;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.handler.codec.http.DefaultFullHttpResponse;
 import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpVersion;
 import io.netty.handler.codec.http.router.KeepAliveWrite;
 import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
 import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
 import org.apache.flink.util.ExceptionUtils;
+import scala.Option;
+import scala.Tuple2;
+import scala.concurrent.Promise;
 
 import java.nio.charset.Charset;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * The Netty channel handler that processes all HTTP requests.
  * This handler takes the path parameters and delegates the work to a {@link RequestHandler}.
  * This handler also deals with setting correct response MIME types and returning
  * proper codes, like OK, NOT_FOUND, or SERVER_ERROR.
  */
-@ChannelHandler.Sharable
 public class RuntimeMonitorHandler extends SimpleChannelInboundHandler<Routed> {
-	
+
 	private static final Charset ENCODING = Charset.forName("UTF-8");
-	
+
 	private final RequestHandler handler;
-	
+
+	private final JobManagerRetriever retriever;
+
+	private final Promise<String> localJobManagerAddressPromise;
+
 	private final String contentType;
-	
-	public RuntimeMonitorHandler(RequestHandler handler) {
-		if (handler == null) {
-			throw new NullPointerException();
-		}
-		this.handler = handler;
+
+	private String localJobManagerAddress;
+
+	public RuntimeMonitorHandler(
+			RequestHandler handler,
+			JobManagerRetriever retriever,
+			Promise<String> localJobManagerAddressPromise) {
+
+		this.handler = checkNotNull(handler);
+		this.retriever = checkNotNull(retriever);
+		this.localJobManagerAddressPromise = checkNotNull(localJobManagerAddressPromise);
 		this.contentType = (handler instanceof RequestHandler.JsonResponse) ? "application/json" : "text/plain";
 	}
-	
+
 	@Override
 	protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
+		if (localJobManagerAddress == null) {
+			localJobManagerAddress = localJobManagerAddressPromise.future().value().get().get();
+		}
+
+		Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
+
+		if (jobManager.isDefined()) {
+			String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
+					localJobManagerAddress, jobManager.get());
+
+			if (redirectAddress != null) {
+				HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, routed.path());
+				KeepAliveWrite.flush(ctx, routed.request(), redirect);
+			}
+			else {
+				respondAsLeader(ctx, routed, jobManager.get()._1());
+			}
+		}
+		else {
+			KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
+		}
+	}
+
+	private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGateway jobManager) {
 		DefaultFullHttpResponse response;
-		
+
 		try {
-			String result = handler.handleRequest(routed.pathParams());
+			String result = handler.handleRequest(routed.pathParams(), jobManager);
 			byte[] bytes = result.getBytes(ENCODING);
-			
+
 			response = new DefaultFullHttpResponse(
 					HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 649cf75..40ab6c1 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -61,6 +61,7 @@ import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.concurrent.Promise;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
@@ -84,7 +85,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 public class WebRuntimeMonitor implements WebMonitor {
 
-	/** By default, all requests to the JobManager have a timeout of 10 seconds */ 
+	/** By default, all requests to the JobManager have a timeout of 10 seconds */
 	public static final FiniteDuration DEFAULT_REQUEST_TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
 
 	/** Logger for web frontend startup / shutdown messages */
@@ -104,13 +105,15 @@ public class WebRuntimeMonitor implements WebMonitor {
 	private final LeaderRetrievalService leaderRetrievalService;
 
 	/** LeaderRetrievalListener which stores the currently leading JobManager and its archive */
-	private final JobManagerArchiveRetriever retriever;
+	private final JobManagerRetriever retriever;
 
 	private final Router router;
 
 	private final int configuredPort;
 
-	private ServerBootstrap bootstrap;
+	private final ServerBootstrap bootstrap;
+
+	private final Promise<String> jobManagerAddressPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
 
 	private Channel serverChannel;
 
@@ -120,10 +123,9 @@ public class WebRuntimeMonitor implements WebMonitor {
 
 
 	public WebRuntimeMonitor(
-				Configuration config,
-				LeaderRetrievalService leaderRetrievalService,
-				ActorSystem actorSystem) throws IOException
-	{
+			Configuration config,
+			LeaderRetrievalService leaderRetrievalService,
+			ActorSystem actorSystem) throws IOException, InterruptedException {
 		this.leaderRetrievalService = checkNotNull(leaderRetrievalService);
 
 		final WebMonitorConfig cfg = new WebMonitorConfig(config);
@@ -175,16 +177,16 @@ public class WebRuntimeMonitor implements WebMonitor {
 		FiniteDuration timeout = AkkaUtils.getTimeout(config);
 		FiniteDuration lookupTimeout = AkkaUtils.getTimeout(config);
 
-		retriever = new JobManagerArchiveRetriever(this, actorSystem, lookupTimeout, timeout);
+		retriever = new JobManagerRetriever(this, actorSystem, lookupTimeout, timeout);
 
-		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder(retriever);
+		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder();
 
 		router = new Router()
 			// config how to interact with this web server
 			.GET("/config", handler(new DashboardConfigHandler(cfg.getRefreshInterval())))
 
 			// the overview - how many task managers, slots, free slots, ...
-			.GET("/overview", handler(new ClusterOverviewHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
+			.GET("/overview", handler(new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)))
 
 			// job manager configuration, log and stdout
 			.GET("/jobmanager/config", handler(new JobManagerConfigHandler(config)))
@@ -192,9 +194,9 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/jobmanager/stdout", new StaticFileServerHandler(outDir))
 
 			// overview over jobs
-			.GET("/joboverview", handler(new CurrentJobsOverviewHandler(retriever, DEFAULT_REQUEST_TIMEOUT, true, true)))
-			.GET("/joboverview/running", handler(new CurrentJobsOverviewHandler(retriever, DEFAULT_REQUEST_TIMEOUT, true, false)))
-			.GET("/joboverview/completed", handler(new CurrentJobsOverviewHandler(retriever, DEFAULT_REQUEST_TIMEOUT, false, true)))
+			.GET("/joboverview", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true)))
+			.GET("/joboverview/running", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, false)))
+			.GET("/joboverview/completed", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, false, true)))
 
 			.GET("/jobs", handler(new CurrentJobIdsHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
 
@@ -215,19 +217,13 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/jobs/:jobid/exceptions", handler(new JobExceptionsHandler(currentGraphs)))
 			.GET("/jobs/:jobid/accumulators", handler(new JobAccumulatorsHandler(currentGraphs)))
 
-			.GET("/taskmanagers", handler(new TaskManagersHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
+			.GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT)))
 			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
 
 			// this handler serves all the static contents
-			.GET("/:*", new StaticFileServerHandler(webRootDir));
-	}
+			.GET("/:*", new StaticFileServerHandler(retriever, jobManagerAddressPromise, webRootDir));
 
-	@Override
-	public void start() throws Exception {
 		synchronized (startupShutdownLock) {
-			if (this.bootstrap != null) {
-				throw new IllegalStateException("The server has already been started");
-			}
 
 			// add shutdown hook for deleting the directory
 			try {
@@ -246,16 +242,16 @@ public class WebRuntimeMonitor implements WebMonitor {
 			}
 
 			ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() {
-	
+
 				@Override
 				protected void initChannel(SocketChannel ch) {
 					Handler handler = new Handler(router);
 
 					ch.pipeline()
-						.addLast(new HttpServerCodec())
-						.addLast(new HttpObjectAggregator(65536))
-						.addLast(new ChunkedWriteHandler())
-						.addLast(handler.name(), handler);
+							.addLast(new HttpServerCodec())
+							.addLast(new HttpObjectAggregator(65536))
+							.addLast(new ChunkedWriteHandler())
+							.addLast(handler.name(), handler);
 				}
 			};
 
@@ -276,7 +272,14 @@ public class WebRuntimeMonitor implements WebMonitor {
 			int port = bindAddress.getPort();
 
 			LOG.info("Web frontend listening at " + address + ':' + port);
+		}
+	}
 
+	@Override
+	public void start(String jobManagerAkkaUrl) throws Exception {
+		LOG.info("Starting with JobManager {} on port {}", jobManagerAkkaUrl, getServerPort());
+		synchronized (startupShutdownLock) {
+			jobManagerAddressPromise.success(jobManagerAkkaUrl);
 			leaderRetrievalService.start(retriever);
 		}
 	}
@@ -294,7 +297,6 @@ public class WebRuntimeMonitor implements WebMonitor {
 				if (bootstrap.group() != null) {
 					bootstrap.group().shutdownGracefully();
 				}
-				this.bootstrap = null;
 			}
 
 			shutdown();
@@ -332,7 +334,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	private static RuntimeMonitorHandler handler(RequestHandler handler) {
-		return new RuntimeMonitorHandler(handler);
+	private RuntimeMonitorHandler handler(RequestHandler handler) {
+		return new RuntimeMonitorHandler(handler, retriever, jobManagerAddressPromise);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
index 51e85b9..944407e 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.webmonitor.files;
 /*****************************************************************************
  * This code is based on the "HttpStaticFileServerHandler" from the
  * Netty project's HTTP server example.
- * 
+ *
  * See http://netty.io and
  * https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
  *****************************************************************************/
@@ -41,12 +41,18 @@ import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.codec.http.router.KeepAliveWrite;
 import io.netty.handler.codec.http.router.Routed;
 import io.netty.util.CharsetUtil;
-
+import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor;
+import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
+import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+import scala.concurrent.Promise;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -61,6 +67,7 @@ import java.util.GregorianCalendar;
 import java.util.Locale;
 import java.util.TimeZone;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
 import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
 import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
@@ -75,55 +82,72 @@ import static io.netty.handler.codec.http.HttpResponseStatus.OK;
 import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 
 /**
- * Simple file server handler that serves requests to web frontend's static files, such as 
+ * Simple file server handler that serves requests to web frontend's static files, such as
  * HTML, CSS, or JS files.
- * 
+ *
  * <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
  * example.</p>
  */
 @ChannelHandler.Sharable
 public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed> {
-	
+
 	/** Default logger, if none is specified */
 	private static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(StaticFileServerHandler.class);
-	
+
 	/** Timezone in which this server answers its "if-modified" requests */
 	private static final TimeZone GMT_TIMEZONE = TimeZone.getTimeZone("GMT");
 
 	/** Date format for HTTP */
 	private static final String HTTP_DATE_FORMAT = "EEE, dd MMM yyyy HH:mm:ss zzz";
-	
+
 	/** Be default, we allow files to be cached for 5 minutes */
 	private static final int HTTP_CACHE_SECONDS = 300;
 
 	// ------------------------------------------------------------------------
 
+	/** JobManager retriever */
+	private final JobManagerRetriever retriever;
+
+	private final Promise<String> localJobManagerAddressPromise;
+
 	/** The path in which the static documents are */
 	private final File rootPath;
 
 	/** The log for all error reporting */
 	private final Logger logger;
 
-	
-	public StaticFileServerHandler(File rootPath) {
-		this(rootPath, DEFAULT_LOGGER);
+	private String localJobManagerAddress;
+
+	public StaticFileServerHandler(
+			JobManagerRetriever retriever,
+			Promise<String> localJobManagerAddressPromise,
+			File rootPath) {
+
+		this(retriever, localJobManagerAddressPromise, rootPath, DEFAULT_LOGGER);
 	}
-	
-	public StaticFileServerHandler(File rootPath, Logger logger) {
-		if (rootPath == null || logger == null) {
-			throw new NullPointerException();
-		}
-		
-		this.rootPath = rootPath;
-		this.logger = logger;
+
+	public StaticFileServerHandler(
+			JobManagerRetriever retriever,
+			Promise<String> localJobManagerAddressPromise,
+			File rootPath,
+			Logger logger) {
+
+		this.retriever = checkNotNull(retriever);
+		this.localJobManagerAddressPromise = localJobManagerAddressPromise;
+		this.rootPath = checkNotNull(rootPath);
+		this.logger = checkNotNull(logger);
 	}
 
 	// ------------------------------------------------------------------------
 	//  Responses to requests
 	// ------------------------------------------------------------------------
-	
+
 	@Override
 	public void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
+		if (localJobManagerAddress == null) {
+			localJobManagerAddress = localJobManagerAddressPromise.future().value().get().get();
+		}
+
 		final HttpRequest request = routed.request();
 		String requestPath = routed.path();
 
@@ -139,6 +163,32 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 			requestPath = "/" + getFileName(rootPath, WebRuntimeMonitor.STDOUT_FILE_PATTERN);
 		}
 
+		Option<Tuple2<ActorGateway, Integer>> jobManager = retriever.getJobManagerGatewayAndWebPort();
+
+		if (jobManager.isDefined()) {
+			// Redirect to leader if necessary
+			String redirectAddress = HandlerRedirectUtils.getRedirectAddress(
+					localJobManagerAddress, jobManager.get());
+
+			if (redirectAddress != null) {
+				HttpResponse redirect = HandlerRedirectUtils.getRedirectResponse(redirectAddress, requestPath);
+				KeepAliveWrite.flush(ctx, routed.request(), redirect);
+			}
+			else {
+				respondAsLeader(ctx, request, requestPath);
+			}
+		}
+		else {
+			KeepAliveWrite.flush(ctx, routed.request(), HandlerRedirectUtils.getUnavailableResponse());
+		}
+	}
+
+	/**
+	 * Response when running with leading JobManager.
+	 */
+	private void respondAsLeader(ChannelHandlerContext ctx, HttpRequest request, String requestPath)
+			throws ParseException, IOException {
+
 		// convert to absolute path
 		final File file = new File(rootPath, requestPath);
 
@@ -157,7 +207,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 				Files.copy(resourceStream, file.toPath());
 			}
 		}
-		
+
 		if (!file.exists() || file.isHidden() || file.isDirectory() || !file.isFile()) {
 			sendError(ctx, NOT_FOUND);
 			return;
@@ -177,7 +227,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 				if (logger.isDebugEnabled()) {
 					logger.debug("Responding 'NOT MODIFIED' for file '" + file.getAbsolutePath() + '\'');
 				}
-				
+
 				sendNotModified(ctx);
 				return;
 			}
@@ -229,15 +279,15 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 			sendError(ctx, INTERNAL_SERVER_ERROR);
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Utilities to encode headers and responses
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Writes a simple  error response message.
-	 * 
-	 * @param ctx The channel context to write the response to.
+	 *
+	 * @param ctx    The channel context to write the response to.
 	 * @param status The response status.
 	 */
 	private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
@@ -279,7 +329,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 	/**
 	 * Sets the "date" and "cache" headers for the HTTP Response.
 	 *
-	 * @param response The HTTP response object.
+	 * @param response    The HTTP response object.
 	 * @param fileToCache File to extract the modification timestamp from.
 	 */
 	private static void setDateAndCacheHeaders(HttpResponse response, File fileToCache) {
@@ -301,7 +351,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
 	 * Sets the content type header for the HTTP Response.
 	 *
 	 * @param response HTTP response
-	 * @param file file to extract content type
+	 * @param file     file to extract content type
 	 */
 	private static void setContentTypeHeader(HttpResponse response, File file) {
 		String mimeType = MimeTypes.getMimeTypeForFileName(file.getName());

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
index d9b4e59..4df387a 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
 import org.apache.flink.runtime.webmonitor.NotFoundException;
 
@@ -33,14 +34,12 @@ public abstract class AbstractExecutionGraphRequestHandler implements RequestHan
 	
 	private final ExecutionGraphHolder executionGraphHolder;
 	
-	
 	public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder) {
 		this.executionGraphHolder = executionGraphHolder;
 	}
-	
-	
+
 	@Override
-	public final String handleRequest(Map<String, String> params) throws Exception {
+	public final String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
 		String jidString = params.get("jobid");
 		if (jidString == null) {
 			throw new RuntimeException("JobId parameter missing");
@@ -54,7 +53,7 @@ public abstract class AbstractExecutionGraphRequestHandler implements RequestHan
 			throw new RuntimeException("Invalid JobID string '" + jidString + "': " + e.getMessage()); 
 		}
 		
-		ExecutionGraph eg = executionGraphHolder.getExecutionGraph(jid);
+		ExecutionGraph eg = executionGraphHolder.getExecutionGraph(jid, jobManager);
 		if (eg == null) {
 			throw new NotFoundException("Could not find job with id " + jid);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
index dde368b..9fcf144 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java
@@ -22,8 +22,6 @@ import com.fasterxml.jackson.core.JsonGenerator;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.webmonitor.RequestStatusOverview;
 import org.apache.flink.runtime.messages.webmonitor.StatusOverview;
-
-import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -31,31 +29,25 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.StringWriter;
 import java.util.Map;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Responder that returns the status of the Flink cluster, such as how many
  * TaskManagers are currently connected, and how many jobs are running.
  */
-public class ClusterOverviewHandler implements  RequestHandler, RequestHandler.JsonResponse {
-
-	private final JobManagerArchiveRetriever retriever;
+public class ClusterOverviewHandler implements RequestHandler, RequestHandler.JsonResponse {
 
 	private final FiniteDuration timeout;
 	
 
-	public ClusterOverviewHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
-		if (retriever == null || timeout == null) {
-			throw new NullPointerException();
-		}
-		this.retriever = retriever;
-		this.timeout = timeout;
+	public ClusterOverviewHandler(FiniteDuration timeout) {
+		this.timeout = checkNotNull(timeout);
 	}
 
 	@Override
-	public String handleRequest(Map<String, String> params) throws Exception {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
 		// we need no parameters, get all requests
 		try {
-			ActorGateway jobManager = retriever.getJobManagerGateway();
-
 			if (jobManager != null) {
 				Future<Object> future = jobManager.ask(RequestStatusOverview.getInstance(), timeout);
 				StatusOverview overview = (StatusOverview) Await.result(future, timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
index 049bd54..06fe34b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java
@@ -24,7 +24,7 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview;
 import org.apache.flink.runtime.messages.webmonitor.RequestJobsWithIDsOverview;
 
-import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
+import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -39,12 +39,12 @@ import java.util.Map;
  */
 public class CurrentJobIdsHandler implements RequestHandler, RequestHandler.JsonResponse {
 
-	private final JobManagerArchiveRetriever retriever;
+	private final JobManagerRetriever retriever;
 
 	private final FiniteDuration timeout;
 
 
-	public CurrentJobIdsHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
+	public CurrentJobIdsHandler(JobManagerRetriever retriever, FiniteDuration timeout) {
 		if (retriever == null || timeout == null) {
 			throw new NullPointerException();
 		}
@@ -53,11 +53,9 @@ public class CurrentJobIdsHandler implements RequestHandler, RequestHandler.Json
 	}
 	
 	@Override
-	public String handleRequest(Map<String, String> params) throws Exception {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
 		// we need no parameters, get all requests
 		try {
-			ActorGateway jobManager = retriever.getJobManagerGateway();
-
 			if (jobManager != null) {
 				Future<Object> future = jobManager.ask(RequestJobsWithIDsOverview.getInstance(), timeout);
 				JobsWithIDsOverview overview = (JobsWithIDsOverview) Await.result(future, timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
index d9bfcb7..3ca0420 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java
@@ -24,8 +24,6 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.webmonitor.JobDetails;
 import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
 import org.apache.flink.runtime.messages.webmonitor.RequestJobDetails;
-
-import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -33,37 +31,33 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.StringWriter;
 import java.util.Map;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Request handler that returns a summary of the job status.
  */
 public class CurrentJobsOverviewHandler implements RequestHandler, RequestHandler.JsonResponse {
 
-	private final JobManagerArchiveRetriever retriever;
-	
 	private final FiniteDuration timeout;
 	
 	private final boolean includeRunningJobs;
 	private final boolean includeFinishedJobs;
 
 	
-	public CurrentJobsOverviewHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout,
-										boolean includeRunningJobs, boolean includeFinishedJobs) {
-		if (retriever == null || timeout == null) {
-			throw new NullPointerException();
-		}
-		this.retriever = retriever;
-		this.timeout = timeout;
+	public CurrentJobsOverviewHandler(
+			FiniteDuration timeout,
+			boolean includeRunningJobs,
+			boolean includeFinishedJobs) {
+
+		this.timeout = checkNotNull(timeout);
 		this.includeRunningJobs = includeRunningJobs;
 		this.includeFinishedJobs = includeFinishedJobs;
 	}
 
 	@Override
-	public String handleRequest(Map<String, String> params) throws Exception {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
 		try {
-			ActorGateway jobManager = retriever.getJobManagerGateway();
-
 			if (jobManager != null) {
-				
 				Future<Object> future = jobManager.ask(
 						new RequestJobDetails(includeRunningJobs, includeFinishedJobs), timeout);
 				

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
index ad72f0a..4027782 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.webmonitor.handlers;
 
 import com.fasterxml.jackson.core.JsonGenerator;
+import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 
 import java.io.StringWriter;
@@ -66,7 +67,7 @@ public class DashboardConfigHandler implements RequestHandler, RequestHandler.Js
 	}
 	
 	@Override
-	public String handleRequest(Map<String, String> params) {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManagerGateway) {
 		return this.configString;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
new file mode 100644
index 0000000..887c46e
--- /dev/null
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.webmonitor.handlers;
+
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.webmonitor.files.MimeTypes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Utilities to extract a redirect address.
+ *
+ * <p>This is necessary at the moment, because many execution graph structures are not serializable.
+ * The proper solution here is to have these serializable and transparently work with the leading
+ * job manager instead of redirecting.
+ */
+public class HandlerRedirectUtils {
+
+	private static final Logger LOG = LoggerFactory.getLogger(HandlerRedirectUtils.class);
+
+	/** Pattern to extract the host from an remote Akka URL */
+	private final static Pattern LeaderAddressHostPattern = Pattern.compile("^.+@(.+):([0-9]+)/user/.+$");
+
+	public static String getRedirectAddress(
+			String localJobManagerAddress,
+			Tuple2<ActorGateway, Integer> leader) throws Exception {
+
+		final String leaderAddress = leader._1().path();
+		final int webMonitorPort = leader._2();
+
+		if (!localJobManagerAddress.equals(leaderAddress)) {
+			// We are not the leader and need to redirect
+			Matcher matcher = LeaderAddressHostPattern.matcher(leaderAddress);
+
+			if (matcher.matches()) {
+				String redirectAddress = String.format("%s:%d", matcher.group(1), webMonitorPort);
+				return redirectAddress;
+			}
+			else {
+				LOG.warn("Unexpected leader address pattern. Cannot extract host.");
+			}
+		}
+
+		return null;
+	}
+
+	public static HttpResponse getRedirectResponse(String redirectAddress, String path) throws Exception {
+		checkNotNull(redirectAddress, "Redirect address");
+		checkNotNull(path, "Path");
+
+		String newLocation = String.format("http://%s%s", redirectAddress, path);
+
+		HttpResponse redirectResponse = new DefaultFullHttpResponse(
+				HttpVersion.HTTP_1_1, HttpResponseStatus.TEMPORARY_REDIRECT);
+		redirectResponse.headers().set(HttpHeaders.Names.LOCATION, newLocation);
+		redirectResponse.headers().set(HttpHeaders.Names.CONTENT_ENCODING, "utf-8");
+		redirectResponse.headers().set(HttpHeaders.Names.CONTENT_LENGTH, 0);
+
+		return redirectResponse;
+	}
+
+	public static HttpResponse getUnavailableResponse() throws UnsupportedEncodingException {
+		String result = "Service temporarily unavailable due to an ongoing leader election. Please refresh.";
+		byte[] bytes = result.getBytes(Charset.forName("UTF-8"));
+
+		HttpResponse unavailableResponse = new DefaultFullHttpResponse(
+				HttpVersion.HTTP_1_1, HttpResponseStatus.SERVICE_UNAVAILABLE, Unpooled.wrappedBuffer(bytes));
+
+		unavailableResponse.headers().set(HttpHeaders.Names.CONTENT_ENCODING, "utf-8");
+		unavailableResponse.headers().set(HttpHeaders.Names.CONTENT_LENGTH, bytes.length);
+		unavailableResponse.headers().set(HttpHeaders.Names.CONTENT_TYPE, MimeTypes.getMimeTypeForExtension("txt"));
+
+		return unavailableResponse;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
index 77314ec..3a0c774 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import com.fasterxml.jackson.core.JsonGenerator;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.instance.ActorGateway;
 
 import java.io.StringWriter;
 import java.util.Map;
@@ -36,7 +37,7 @@ public class JobManagerConfigHandler implements RequestHandler, RequestHandler.J
 	}
 
 	@Override
-	public String handleRequest(Map<String, String> params) throws Exception {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManagerGateway) throws Exception {
 		StringWriter writer = new StringWriter();
 		JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
index 4ffb9d9..53d1179 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.webmonitor.handlers;
 
+import org.apache.flink.runtime.instance.ActorGateway;
+
 import java.util.Map;
 
 public interface RequestHandler {
@@ -25,15 +27,15 @@ public interface RequestHandler {
 	/**
 	 * This interface marks handlers that return JSON data.
 	 */
-	public static interface JsonResponse {}
+	interface JsonResponse {}
 
 	/**
 	 * This interface marks handlers that return plain text data.
 	 */
-	public static interface TextResponse {}
+	interface TextResponse {}
 	
 	
 	// --------------------------------------------------------------------------------------------
 
-	String handleRequest(Map<String, String> params) throws Exception;
+	String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
index dbdb439..0becb6a 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
@@ -23,7 +23,6 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import org.apache.flink.runtime.messages.JobManagerMessages.RegisteredTaskManagers;
 import org.apache.flink.runtime.messages.JobManagerMessages.TaskManagerInstance;
 import org.apache.flink.util.StringUtils;
@@ -36,27 +35,21 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-public class TaskManagersHandler implements  RequestHandler, RequestHandler.JsonResponse {
+import static com.google.common.base.Preconditions.checkNotNull;
 
-	private final JobManagerArchiveRetriever retriever;
+public class TaskManagersHandler implements RequestHandler, RequestHandler.JsonResponse {
 
 	private final FiniteDuration timeout;
 
 	public static final String TASK_MANAGER_ID_KEY = "taskmanagerid";
-
-	public TaskManagersHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
-		if (retriever == null || timeout == null) {
-			throw new NullPointerException();
-		}
-		this.retriever = retriever;
-		this.timeout = timeout;
+	
+	public TaskManagersHandler(FiniteDuration timeout) {
+		this.timeout = checkNotNull(timeout);
 	}
 
 	@Override
-	public String handleRequest(Map<String, String> params) throws Exception {
+	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
 		try {
-			ActorGateway jobManager = retriever.getJobManagerGateway();
-
 			if (jobManager != null) {
 				// whether one task manager's metrics are requested, or all task manager, we
 				// return them in an array. This avoids unnecessary code complexity.
@@ -117,7 +110,8 @@ public class TaskManagersHandler implements  RequestHandler, RequestHandler.Json
 
 				gen.close();
 				return writer.toString();
-			} else {
+			}
+			else {
 				throw new Exception("No connection to the leading JobManager.");
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/css/index.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/css/index.css b/flink-runtime-web/src/main/resources/web/css/index.css
new file mode 100644
index 0000000..43f7d98
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/css/index.css
@@ -0,0 +1,596 @@
+/*
+ * 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.
+ */
+#sidebar {
+  overflow: hidden;
+  position: fixed;
+  left: -250px;
+  top: 0;
+  bottom: 0;
+  height: 100%;
+  width: 250px;
+  background: #151515;
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+  -webkit-box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
+  box-shadow: inset -10px 0px 10px rgba(0,0,0,0.2);
+}
+#sidebar.sidebar-visible {
+  left: 0;
+}
+#sidebar .logo {
+  width: auto;
+  height: 22px;
+}
+#sidebar .logo img {
+  display: inline-block;
+}
+#sidebar .navbar-static-top {
+  overflow: hidden;
+  height: 51px;
+}
+#sidebar .navbar-static-top .navbar-header {
+  width: 100%;
+}
+#sidebar .navbar-brand.navbar-brand-text {
+  font-size: 14px;
+  font-weight: bold;
+  color: #fff;
+  padding-left: 0;
+}
+#sidebar .nav > li > a {
+  color: #aaa;
+  margin-bottom: 1px;
+}
+#sidebar .nav > li > a:hover,
+#sidebar .nav > li > a:focus {
+  background-color: rgba(40,40,40,0.5);
+}
+#sidebar .nav > li > a.active {
+  background-color: rgba(100,100,100,0.5);
+}
+#content {
+  background-color: #fff;
+  overflow: hidden;
+  margin-left: 0;
+  padding-top: 70px;
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+}
+#content .navbar-main,
+#content .navbar-main-additional {
+  -webkit-transition: 400ms;
+  -moz-transition: 400ms;
+  -o-transition: 400ms;
+  -ms-transition: 400ms;
+  transition: 400ms;
+}
+#content .navbar-main-additional {
+  margin-top: 51px;
+  border-bottom: none;
+  padding: 0 20px;
+}
+#content .navbar-main-additional .nav-tabs {
+  margin: 0 -20px;
+  padding: 0 20px;
+}
+#content .navbar-secondary-additional {
+  border: none;
+  padding: 0 20px;
+  margin-bottom: 0;
+}
+#content .navbar-secondary-additional .nav-tabs {
+  margin: 0 -20px;
+}
+#content.sidebar-visible {
+  margin-left: 250px;
+}
+#content.sidebar-visible .navbar-main,
+#content.sidebar-visible .navbar-main-additional {
+  left: 250px;
+}
+#content #fold-button {
+  display: inline-block;
+  margin-left: 20px;
+}
+#content #content-inner {
+  padding: 0px 20px 20px 20px;
+}
+#content #content-inner.has-navbar-main-additional {
+  padding-top: 42px;
+}
+.page-header {
+  margin: 0 0 20px 0;
+}
+.nav > li > a,
+.nav > li > a:hover,
+.nav > li > a:focus {
+  color: #aaa;
+  background-color: transparent;
+  border-bottom: 2px solid transparent;
+}
+.nav > li.active > a,
+.nav > li.active > a:hover,
+.nav > li.active > a:focus {
+  color: #000;
+  border-bottom: 2px solid #000;
+}
+.nav.nav-tabs {
+  margin-bottom: 20px;
+}
+.table .table {
+  background-color: transparent;
+}
+.table th {
+  font-weight: normal;
+  color: #999;
+}
+.table td.td-long {
+  width: 20%;
+  white-space: pre-wrap;
+  white-space: -moz-pre-wrap;
+  white-space: -pre-wrap;
+  white-space: -o-pre-wrap;
+  word-wrap: break-word;
+}
+.table.table-clickable tr {
+  cursor: pointer;
+}
+.table.table-inner {
+  background-color: transparent;
+}
+.table.table-properties {
+  table-layout: fixed;
+  white-space: nowrap;
+}
+.table.table-properties td {
+  width: 50%;
+  white-space: nowrap;
+  overflow: hidden;
+  -o-text-overflow: ellipsis;
+  text-overflow: ellipsis;
+}
+.table.table-body-hover > tbody {
+  border-top: none;
+  border-left: 2px solid transparent;
+}
+.table.table-body-hover > tbody.active {
+  border-left: 2px solid #000;
+}
+.table.table-body-hover > tbody:hover td:not(.tab-column),
+.table.table-body-hover > tbody.active td:not(.tab-column) {
+  background-color: #f0f0f0;
+}
+.table.table-body-hover > tbody:hover td.tab-column li.active,
+.table.table-body-hover > tbody.active td.tab-column li.active {
+  background-color: #f0f0f0;
+}
+.table.table-activable th.tab-column,
+.table.table-activable td.tab-column {
+  border-top: none;
+  width: 47px;
+}
+.table.table-activable td.tab-column {
+  border-right: 1px solid #ddd;
+}
+.table.table-activable td {
+  position: relative;
+}
+.table .small-label {
+  text-transform: uppercase;
+  font-size: 13px;
+  color: #999;
+}
+.panel.panel-dashboard .huge {
+  font-size: 28px;
+}
+.panel.panel-lg {
+  font-size: 16px;
+}
+.panel.panel-lg .badge {
+  font-size: 14px;
+}
+.navbar-secondary {
+  overflow: auto;
+}
+.navbar-main .navbar-title,
+.navbar-secondary .navbar-title,
+.navbar-main-additional .navbar-title,
+.panel.panel-multi .navbar-title,
+.navbar-secondary-additional .navbar-title,
+.navbar-main .panel-title,
+.navbar-secondary .panel-title,
+.navbar-main-additional .panel-title,
+.panel.panel-multi .panel-title,
+.navbar-secondary-additional .panel-title {
+  float: left;
+  font-size: 18px;
+  padding: 12px 20px 13px 10px;
+  color: #333;
+  display: inline-block;
+}
+.navbar-main .navbar-info,
+.navbar-secondary .navbar-info,
+.navbar-main-additional .navbar-info,
+.panel.panel-multi .navbar-info,
+.navbar-secondary-additional .navbar-info,
+.navbar-main .panel-info,
+.navbar-secondary .panel-info,
+.navbar-main-additional .panel-info,
+.panel.panel-multi .panel-info,
+.navbar-secondary-additional .panel-info {
+  float: left;
+  font-size: 14px;
+  padding: 15px 15px 15px 15px;
+  color: #999;
+  display: inline-block;
+  border-right: 1px solid #e7e7e7;
+  overflow: hidden;
+}
+.navbar-main .navbar-info .overflow,
+.navbar-secondary .navbar-info .overflow,
+.navbar-main-additional .navbar-info .overflow,
+.panel.panel-multi .navbar-info .overflow,
+.navbar-secondary-additional .navbar-info .overflow,
+.navbar-main .panel-info .overflow,
+.navbar-secondary .panel-info .overflow,
+.navbar-main-additional .panel-info .overflow,
+.panel.panel-multi .panel-info .overflow,
+.navbar-secondary-additional .panel-info .overflow {
+  position: absolute;
+  display: block;
+  -o-text-overflow: ellipsis;
+  text-overflow: ellipsis;
+  overflow: hidden;
+  height: 22px;
+  line-height: 22px;
+  vertical-align: middle;
+}
+.navbar-main .navbar-info.first,
+.navbar-secondary .navbar-info.first,
+.navbar-main-additional .navbar-info.first,
+.panel.panel-multi .navbar-info.first,
+.navbar-secondary-additional .navbar-info.first,
+.navbar-main .panel-info.first,
+.navbar-secondary .panel-info.first,
+.navbar-main-additional .panel-info.first,
+.panel.panel-multi .panel-info.first,
+.navbar-secondary-additional .panel-info.first {
+  border-left: 1px solid #e7e7e7;
+}
+.navbar-main .navbar-info.last,
+.navbar-secondary .navbar-info.last,
+.navbar-main-additional .navbar-info.last,
+.panel.panel-multi .navbar-info.last,
+.navbar-secondary-additional .navbar-info.last,
+.navbar-main .panel-info.last,
+.navbar-secondary .panel-info.last,
+.navbar-main-additional .panel-info.last,
+.panel.panel-multi .panel-info.last,
+.navbar-secondary-additional .panel-info.last {
+  border-right: none;
+}
+.panel.panel-multi .panel-heading {
+  padding: 0;
+}
+.panel.panel-multi .panel-heading .panel-info.thin {
+  padding: 8px 10px;
+}
+.panel.panel-multi .panel-body {
+  padding: 10px;
+  background-color: #fdfdfd;
+  color: #999;
+  font-size: 13px;
+}
+.panel.panel-multi .panel-body.clean {
+  color: inherit;
+  font-size: inherit;
+}
+.navbar-main-additional,
+.navbar-secondary-additional {
+  min-height: 40px;
+  background-color: #fdfdfd;
+}
+.navbar-main-additional .navbar-info,
+.navbar-secondary-additional .navbar-info {
+  font-size: 13px;
+  padding: 10px 15px 10px 15px;
+}
+.nav-top-affix.affix {
+  width: 100%;
+  top: 50px;
+  margin-left: -20px;
+  padding-left: 20px;
+  margin-right: -20px;
+  padding-right: 20px;
+  background-color: #fff;
+  z-index: 1;
+}
+.badge-default[href]:hover,
+.badge-default[href]:focus {
+  background-color: #808080;
+}
+.badge-primary {
+  background-color: #428bca;
+}
+.badge-primary[href]:hover,
+.badge-primary[href]:focus {
+  background-color: #3071a9;
+}
+.badge-success {
+  background-color: #5cb85c;
+}
+.badge-success[href]:hover,
+.badge-success[href]:focus {
+  background-color: #449d44;
+}
+.badge-info {
+  background-color: #5bc0de;
+}
+.badge-info[href]:hover,
+.badge-info[href]:focus {
+  background-color: #31b0d5;
+}
+.badge-warning {
+  background-color: #f0ad4e;
+}
+.badge-warning[href]:hover,
+.badge-warning[href]:focus {
+  background-color: #ec971f;
+}
+.badge-danger {
+  background-color: #d9534f;
+}
+.badge-danger[href]:hover,
+.badge-danger[href]:focus {
+  background-color: #c9302c;
+}
+.indicator {
+  display: inline-block;
+  margin-right: 15px;
+}
+.indicator.indicator-primary {
+  color: #428bca;
+}
+.indicator.indicator-success {
+  color: #5cb85c;
+}
+.indicator.indicator-info {
+  color: #5bc0de;
+}
+.indicator.indicator-warning {
+  color: #f0ad4e;
+}
+.indicator.indicator-danger {
+  color: #d9534f;
+}
+pre.exception {
+  border: none;
+  background-color: transparent;
+  padding: 0;
+  margin: 0;
+}
+.nav-tabs.tabs-vertical {
+  position: absolute;
+  left: 0;
+  top: 0;
+  border-bottom: none;
+  z-index: 100;
+}
+.nav-tabs.tabs-vertical li {
+  float: none;
+  margin-bottom: 0;
+  margin-right: -1px;
+}
+.nav-tabs.tabs-vertical li > a {
+  margin-right: 0;
+  -webkit-border-radius: 0;
+  border-radius: 0;
+  border-bottom: none;
+  border-left: 2px solid transparent;
+}
+.nav-tabs.tabs-vertical li > a:hover,
+.nav-tabs.tabs-vertical li > a:focus {
+  border-bottom: none;
+  border-left: 2px solid #000;
+}
+.nav-tabs.tabs-vertical li.active > a {
+  border-bottom: none;
+  border-left: 2px solid #000;
+}
+.navbar-main .navbar-title,
+.navbar-secondary .navbar-title,
+.navbar-main-additional .navbar-title,
+.navbar-secondary-additional .navbar-title {
+  padding: 12px 20px 13px 20px;
+}
+livechart {
+  width: 30%;
+  height: 30%;
+  text-align: center;
+}
+.canvas-wrapper {
+  border: 1px solid #ddd;
+  position: relative;
+  margin-bottom: 20px;
+}
+.canvas-wrapper .main-canvas {
+  height: 400px;
+  overflow: hidden;
+}
+.canvas-wrapper .main-canvas .zoom-buttons {
+  position: absolute;
+  top: 10px;
+  right: 10px;
+}
+.label-group .label {
+  display: inline-block;
+  width: 2em;
+  padding-left: 0.1em;
+  padding-right: 0.1em;
+  margin: 0;
+  border-right: 1px solid #fff;
+  -webkit-border-radius: 0;
+  border-radius: 0;
+}
+.label-group .label.label-black {
+  background-color: #000;
+}
+svg.graph {
+  overflow: hidden;
+}
+svg.graph g.type-TK > rect {
+  fill: #00ffd0;
+}
+svg.graph text {
+  font-weight: 300;
+  font-size: 14px;
+}
+svg.graph .node {
+  cursor: pointer;
+}
+svg.graph .node > rect {
+  stroke: #999;
+  stroke-width: 5px;
+  fill: #fff;
+  margin: 0;
+  padding: 0;
+}
+svg.graph .node[active] > rect {
+  fill: #eee;
+}
+svg.graph .node.node-mirror > rect {
+  stroke: #a8a8a8;
+}
+svg.graph .node.node-iteration > rect {
+  stroke: #cd3333;
+}
+svg.graph .node.node-source > rect {
+  stroke: #4ce199;
+}
+svg.graph .node.node-sink > rect {
+  stroke: #e6ec8b;
+}
+svg.graph .node.node-normal > rect {
+  stroke: #3fb6d8;
+}
+svg.graph .node h4 {
+  color: #000;
+}
+svg.graph .node h5 {
+  color: #999;
+}
+svg.graph .edgeLabel rect {
+  fill: #fff;
+}
+svg.graph .edgePath path {
+  stroke: #333;
+  stroke-width: 2px;
+  fill: #333;
+}
+svg.graph .label {
+  color: #777;
+  margin: 0;
+}
+svg.graph .edge-label {
+  font-size: 14px;
+}
+svg.graph .node-label {
+  display: block;
+  margin: 0;
+  text-decoration: none;
+}
+.timeline {
+  overflow: hidden;
+}
+.timeline-canvas {
+  overflow: hidden;
+  padding: 10px;
+}
+.timeline-canvas .bar-container {
+  overflow: hidden;
+}
+.timeline-canvas .timeline-insidelabel,
+.timeline-canvas .timeline-series {
+  cursor: pointer;
+}
+.timeline-canvas.secondary .timeline-insidelabel,
+.timeline-canvas.secondary .timeline-series {
+  cursor: auto;
+}
+.qtip-timeline-bar {
+  font-size: 14px;
+  line-height: 1.4;
+}
+@media (min-width: 1024px) and (max-width: 1279px) {
+  #sidebar {
+    left: 0;
+    width: 160px;
+  }
+  #sidebar .navbar-static-top .navbar-brand-text {
+    display: none;
+  }
+  #content {
+    margin-left: 160px;
+  }
+  #content #fold-button {
+    display: none;
+  }
+  #content .navbar-main,
+  #content .navbar-main-additional {
+    left: 160px;
+  }
+  .table td.td-long {
+    width: 20%;
+  }
+}
+@media (min-width: 1280px) {
+  #sidebar {
+    left: 0;
+  }
+  #content {
+    margin-left: 250px;
+  }
+  #content #fold-button {
+    display: none;
+  }
+  #content .navbar-main,
+  #content .navbar-main-additional {
+    left: 250px;
+  }
+  .table td.td-long {
+    width: 30%;
+  }
+}
+#total-mem {
+  background-color: #7cb5ec;
+}
+#heap-mem {
+  background-color: #434348;
+}
+#non-heap-mem {
+  background-color: #90ed7d;
+}
+a.show-pointer {
+  cursor: pointer;
+}


[33/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
new file mode 100644
index 0000000..da6b75b
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobmanager/config.html
@@ -0,0 +1,33 @@
+
+<!--
+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.
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th>Key</th>
+      <th>Value</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="entry in jobmanager.config | orderBy: 'key'">
+      <td>{{entry.key}}</td>
+      <td>{{entry.value}}</td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
new file mode 100644
index 0000000..02c2f47
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobmanager/index.html
@@ -0,0 +1,33 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Job Manager</div>
+</nav>
+<nav class="navbar navbar-default navbar-fixed-top navbar-main-additional">
+  <ul class="nav nav-tabs">
+    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
+    <li ui-sref-active="active"><a ui-sref=".log">Logs</a></li>
+    <li ui-sref-active="active"><a ui-sref=".stdout">Stdout</a></li>
+  </ul>
+</nav>
+<div id="content-inner" class="has-navbar-main-additional">
+  <div ui-view="details"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html b/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
new file mode 100644
index 0000000..df6a817
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobmanager/stdout.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">
+        <div class="row">
+          <div class="col-xs-10">Job Manager Output</div>
+          <div class="col-xs-1 text-right"><a ng-click="reloadData()" class="show-pointer"><i class="fa fa-refresh"></i></a></div>
+          <div class="col-xs-1 text-left"><a href="jobmanager/stdout"><i class="fa fa-download"></i></a></div>
+        </div>
+      </th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td colspan="2">
+        <pre>{{jobmanager.stdout}}</pre>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
new file mode 100644
index 0000000..b76278d
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
@@ -0,0 +1,53 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Completed Jobs</div>
+</nav>
+<div id="content-inner">
+  <table class="table table-hover table-clickable">
+    <thead>
+      <tr>
+        <th>Start Time</th>
+        <th>End Time</th>
+        <th>Duration</th>
+        <th>Job Name</th>
+        <th>Job ID</th>
+        <th>Tasks</th>
+        <th>Status</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job.duration}} ms</td>
+        <td>{{job.name}}</td>
+        <td>{{job.jid}}</td>
+        <td class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+        </td>
+        <td> 
+          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+        </td>
+      </tr>
+    </tbody>
+  </table>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
new file mode 100644
index 0000000..a7a5d9d
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.config.html
@@ -0,0 +1,57 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table ng-if="job['execution-config']" class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">Execution configuration</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Execution mode</td>
+      <td>{{ job['execution-config']['execution-mode'] }}</td>
+    </tr>
+    <tr>
+      <td>Max. number of execution retries</td>
+      <td>{{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }}</td>
+    </tr>
+    <tr>
+      <td>Job parallelism</td>
+      <td>{{ job['execution-config']['job-parallelism'] === -1 ? 'auto' : job['execution-config']['job-parallelism'] }}</td>
+    </tr>
+    <tr>
+      <td>Object reuse mode</td>
+      <td>{{ job['execution-config']['object-reuse-mode'] }}</td>
+    </tr>
+  </tbody>
+</table>
+<table ng-if="job['execution-config']['user-config']" class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">User configuration</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="property in job['execution-config']['user-config']">
+      <td>{{property.name}}</td>
+      <td table-property="table-property" value="property.value"></td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/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
new file mode 100644
index 0000000..a5f6676
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.exceptions.html
@@ -0,0 +1,38 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div ng-if="exceptions['root-exception']" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">Root exception</div>
+  </div>
+  <div class="panel-body">
+    <pre class="exception">{{ exceptions['root-exception'] }}</pre>
+  </div>
+</div>
+<div ng-repeat="exception in exceptions['all-exceptions']" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <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>
+  <div class="panel-body">
+    <pre class="exception">{{ exception.exception }}</pre>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
new file mode 100644
index 0000000..9d3e171
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
@@ -0,0 +1,48 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">
+    <indicator-primary status="{{job.state}}"></indicator-primary>{{ job.name }}
+  </div>
+  <div class="navbar-info first last hidden-xs hidden-sm">{{ job.jid }}</div>
+  <div class="navbar-info first last">
+    <div class="label-group">
+      <bs-label status="{{status}}" ng-repeat="(status, value) in job['status-counts']">{{value}}</bs-label>
+    </div>
+  </div>
+  <div class="navbar-info first last hidden-xs hidden-sm">{{ job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}<span ng-if="job['end-time'] &gt; -1">
+      - 
+      {{ job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></div>
+  <div ng-if="job.duration &gt; -1" class="navbar-info last first">{{job.duration}} ms</div>
+</nav>
+<nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
+  <ul class="nav nav-tabs">
+    <li ui-sref-active="active"><a ui-sref=".plan.overview">Plan</a></li>
+    <li ui-sref-active="active"><a ui-sref=".statistics">Job Accumulators / Statistics</a></li>
+    <li ui-sref-active="active"><a ui-sref=".timeline">Timeline</a></li>
+    <li ui-sref-active="active"><a ui-sref=".exceptions">Exceptions</a></li>
+    <li ui-sref-active="active"><a ui-sref=".properties">Properties</a></li>
+    <li ui-sref-active="active"><a ui-sref=".config">Configuration</a></li>
+  </ul>
+</nav>
+<div id="content-inner" class="has-navbar-main-additional">
+  <div ui-view="details"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
new file mode 100644
index 0000000..f2c4143
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.html
@@ -0,0 +1,31 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div class="canvas-wrapper">
+  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
+</div>
+<div ng-if="plan" class="panel panel-default panel-multi">
+  <nav class="navbar navbar-default navbar-secondary-additional">
+    <ul class="nav nav-tabs">
+      <li ui-sref-active="active"><a ui-sref=".overview({nodeid: nodeid})">Overview</a></li>
+      <li ui-sref-active="active"><a ui-sref=".accumulators({nodeid: nodeid})">Accumulators</a></li>
+    </ul>
+  </nav>
+  <div ui-view="node-details" class="panel-body clean"></div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
new file mode 100644
index 0000000..8de3921
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.accumulators.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table class="table table-body-hover table-clickable table-activable">
+  <thead>
+    <tr>
+      <th>Name</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="v.id == nodeid || changeNode(v.id)">
+    <tr ng-if="v.type == 'regular'">
+      <td>{{ v.name | humanizeText }}</td>
+      <td> 
+        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
+      </td>
+    </tr>
+    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
+      <td colspan="10">
+        <div ng-include=" 'partials/jobs/job.plan.node.accumulators.html' "></div>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
new file mode 100644
index 0000000..1706d3e
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.overview.html
@@ -0,0 +1,60 @@
+
+<!--
+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.
+
+-->
+<table class="table table-body-hover table-clickable table-activable">
+  <thead>
+    <tr>
+      <th>Start Time</th>
+      <th>End Time</th>
+      <th>Duration</th>
+      <th>Name</th>
+      <th>Bytes read</th>
+      <th>Records read</th>
+      <th>Bytes written</th>
+      <th>Records written</th>
+      <th>Tasks</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody ng-repeat="v in job.vertices" ng-class="{ active: v.id == nodeid }" ng-click="changeNode(v.id)">
+    <tr ng-if="v.type == 'regular'">
+      <td><span ng-if="v['start-time'] &gt; -1">{{ v['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="v['end-time'] &gt; -1">{{ v['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="v.duration &gt; -1">{{ v.duration }} ms</span></td>
+      <td class="td-long">{{ v.name | humanizeText }}</td>
+      <td>{{ v.metrics['read-bytes'] }}</td>
+      <td>{{ v.metrics['read-records'] }}</td>
+      <td>{{ v.metrics['write-bytes'] }}</td>
+      <td>{{ v.metrics['write-records'] }}</td>
+      <td>
+        <div class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(index, status) in stateList">{{v.tasks[status]}}</bs-label>
+        </div>
+      </td>
+      <td> 
+        <bs-label status="{{v.status}}">{{v.status}}</bs-label>
+      </td>
+    </tr>
+    <tr ng-if="nodeid &amp;&amp; v.id == nodeid">
+      <td colspan="10">
+        <div ng-include=" 'partials/jobs/job.plan.node.subtasks.html' "></div>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
new file mode 100644
index 0000000..e7dcf2c
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.accumulators.html
@@ -0,0 +1,68 @@
+
+<!--
+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.
+
+-->
+<div ng-if="accumulators.length == 0">
+  <p><i>No accumulators</i></p>
+</div>
+<div ng-if="accumulators &amp;&amp; accumulators.length &gt; 0">
+  <table class="table table-hover table-clickable table-activable table-inner">
+    <thead>
+      <tr>
+        <th>Name</th>
+        <th>Type</th>
+        <th>Value</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="accumulator in accumulators">
+        <td width="30%">{{ accumulator.name }}</td>
+        <td width="30%">{{ accumulator.type }}</td>
+        <td width="30%">{{ accumulator.value }}</td>
+      </tr>
+    </tbody>
+  </table>
+  <div ng-if="!nodeUnfolded"><a ng-click="toggleFold()" class="btn btn-default">
+      Show subtasks
+       <i class="fa fa-chevron-down"></i></a><a ng-click="deactivateNode(); $event.stopPropagation()" title="Fold" class="btn btn-default pull-right"><i class="fa fa-chevron-up"></i></a></div>
+  <div ng-if="nodeUnfolded &amp;&amp; subtaskAccumulators &amp;&amp; subtaskAccumulators.length &gt; 0"><a ng-click="toggleFold()" class="btn btn-default">
+      Hide subtasks
+       <i class="fa fa-chevron-up"></i></a>
+    <table class="table table-hover table-clickable table-activable table-inner">
+      <thead>
+        <tr>
+          <th>Name</th>
+          <th>Type</th>
+          <th>Value</th>
+        </tr>
+      </thead>
+      <tbody ng-if="subtask['user-accumulators'] &amp;&amp; subtask['user-accumulators'].length &gt; 0" ng-repeat="subtask in subtaskAccumulators">
+        <tr>
+          <td colwidth="3">
+            <div class="small-label">({{ subtask.subtask }}) {{ subtask.host }}, attempt: {{ subtask.attempt + 1 }}</div>
+          </td>
+        </tr>
+        <tr ng-repeat="accumulator in subtask['user-accumulators']">
+          <td width="30%">{{ accumulator.name }}</td>
+          <td width="30%">{{ accumulator.type }}</td>
+          <td width="30%">{{ accumulator.value }}</td>
+        </tr>
+      </tbody>
+    </table>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
new file mode 100644
index 0000000..40b16bc
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node.subtasks.html
@@ -0,0 +1,52 @@
+
+<!--
+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.
+
+
+-->
+<table ng-if="subtasks" class="table table-hover table-clickable table-activable table-inner">
+  <thead>
+    <tr>
+      <th>Start Time</th>
+      <th>End Time</th>
+      <th>Duration</th>
+      <th>Bytes read</th>
+      <th>Records read</th>
+      <th>Bytes written</th>
+      <th>Records written</th>
+      <th>Attempt</th>
+      <th>Host</th>
+      <th>Status</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr ng-repeat="subtask in subtasks">
+      <td><span ng-if="subtask['start-time'] &gt; -1">{{ subtask['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="subtask['end-time'] &gt; -1">{{ subtask['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</span></td>
+      <td><span ng-if="subtask.duration &gt; -1">{{ subtask.duration }} ms</span></td>
+      <td><span ng-if="subtask.metrics['read-bytes'] &gt; -1">{{ subtask.metrics['read-bytes'] }}</span></td>
+      <td><span ng-if="subtask.metrics['read-records'] &gt; -1">{{ subtask.metrics['read-records'] }}</span></td>
+      <td><span ng-if="subtask.metrics['write-bytes'] &gt; -1">{{ subtask.metrics['write-bytes'] }}</span></td>
+      <td><span ng-if="subtask.metrics['write-records'] &gt; -1">{{ subtask.metrics['write-records'] }}</span></td>
+      <td>{{ subtask.attempt + 1 }}</td>
+      <td>{{ subtask.host }}</td>
+      <td> 
+        <bs-label status="{{subtask.status}}">{{subtask.status}}</bs-label>
+      </td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
new file mode 100644
index 0000000..907afd3
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.properties.html
@@ -0,0 +1,140 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<div class="canvas-wrapper">
+  <div job-plan="job-plan" plan="plan" jobid="{{jobid}}" set-node="changeNode(nodeid)" class="main-canvas"></div>
+</div>
+<div ng-if="node" class="panel panel-default">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">{{ node.description | humanizeText }}</div>
+  </div>
+  <div class="panel-body clean">
+    <div class="row">
+      <div class="col-sm-6 col-md-4">
+        <table ng-if="node.optimizer_properties.global_properties" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Global Data Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.global_properties">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+        <table ng-if="node.optimizer_properties.local_properties" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Local Data Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.local_properties">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+        <div class="visible-xs visible-sm">
+          <table class="table table-properties">
+            <thead>
+              <tr>
+                <th colspan="2">Properties</th>
+              </tr>
+            </thead>
+            <tbody>
+              <tr>
+                <td>Operator</td>
+                <td table-property="table-property" value="node.operator_strategy"></td>
+              </tr>
+              <tr>
+                <td>Parallelism</td>
+                <td table-property="table-property" value="node.parallelism"></td>
+              </tr>
+            </tbody>
+          </table>
+        </div>
+      </div>
+      <div class="hidden-sm col-md-4">
+        <table class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Properties</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr>
+              <td>Operator</td>
+              <td table-property="table-property" value="node.operator_strategy"></td>
+            </tr>
+            <tr>
+              <td>Parallelism</td>
+              <td table-property="table-property" value="node.parallelism"></td>
+            </tr>
+          </tbody>
+        </table>
+        <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Size Estimates</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.estimates">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+      </div>
+      <div class="col-sm-6 col-md-4">
+        <div class="visible-xs visible-sm">
+          <table ng-if="node.optimizer_properties.estimates" class="table table-properties">
+            <thead>
+              <tr>
+                <th colspan="2">Size Estimates</th>
+              </tr>
+            </thead>
+            <tbody>
+              <tr ng-repeat="property in node.optimizer_properties.estimates">
+                <td>{{property.name}}</td>
+                <td table-property="table-property" value="property.value"></td>
+              </tr>
+            </tbody>
+          </table>
+        </div>
+        <table ng-if="node.optimizer_properties.costs" class="table table-properties">
+          <thead>
+            <tr>
+              <th colspan="2">Cost Estimates</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr ng-repeat="property in node.optimizer_properties.costs">
+              <td>{{property.name}}</td>
+              <td table-property="table-property" value="property.value"></td>
+            </tr>
+          </tbody>
+        </table>
+      </div>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
new file mode 100644
index 0000000..951cc1c
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.statistics.html
@@ -0,0 +1,40 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<table class="table table-properties">
+  <thead>
+    <tr>
+      <th colspan="2">Some statistics</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Operator</td>
+      <td>1</td>
+    </tr>
+    <tr>
+      <td>Parallelism</td>
+      <td>2</td>
+    </tr>
+    <tr>
+      <td>Subtasks-per-instance</td>
+      <td>3</td>
+    </tr>
+  </tbody>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
new file mode 100644
index 0000000..2f22576
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.html
@@ -0,0 +1,23 @@
+
+<!--
+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.
+
+-->
+<div class="canvas-wrapper">
+  <div timeline="timeline" vertices="vertices" jobid="jobid" class="timeline-canvas"></div>
+</div>
+<div ui-view="vertex"></div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
new file mode 100644
index 0000000..1a4bd06
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.timeline.vertex.html
@@ -0,0 +1,30 @@
+
+<!--
+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.
+
+-->
+<div ng-if="vertex" class="panel panel-default panel-multi">
+  <div class="panel-heading clearfix">
+    <div class="panel-title">{{ vertex.groupvertex.groupvertexname | humanizeText }}</div>
+  </div>
+  <div class="panel-body">
+    <div class="canvas-wrapper">
+      <div vertex="vertex" data="vertex" class="timeline-canvas"></div>
+    </div>
+    <div id="timeline1"></div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
new file mode 100644
index 0000000..e175d07
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
@@ -0,0 +1,53 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Running Jobs</div>
+</nav>
+<div id="content-inner">
+  <table class="table table-hover table-clickable">
+    <thead>
+      <tr>
+        <th>Start Time</th>
+        <th>End Time</th>
+        <th>Duration</th>
+        <th>Job Name</th>
+        <th>Job ID</th>
+        <th>Tasks</th>
+        <th>Status</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+        <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+        <td>{{job.duration}} ms</td>
+        <td>{{job.name}}</td>
+        <td>{{job.jid}}</td>
+        <td class="label-group">
+          <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+        </td>
+        <td> 
+          <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+        </td>
+      </tr>
+    </tbody>
+  </table>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/overview.html b/flink-runtime-web/web-dashboard/web/partials/overview.html
new file mode 100644
index 0000000..ec3c580
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/overview.html
@@ -0,0 +1,147 @@
+
+<!--
+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.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Overview</div>
+</nav>
+<div id="content-inner">
+  <div class="row">
+    <div class="col-md-6">
+      <div class="panel panel-default panel-dashboard">
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-tasks fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview.taskmanagers}}</div>
+              <div>Task Managers</div>
+            </div>
+          </div>
+        </div>
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-folder fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview["slots-total"]}}</div>
+              <div>Task Slots</div>
+            </div>
+          </div>
+        </div>
+        <div class="panel-heading">
+          <div class="row">
+            <div class="col-xs-3"><i class="fa fa-folder-o fa-3x"></i></div>
+            <div class="col-xs-9 text-right">
+              <div class="huge">{{overview["slots-available"]}}</div>
+              <div>Available Task Slots</div>
+            </div>
+          </div>
+        </div>
+      </div>
+    </div>
+    <div class="col-md-6">
+      <div class="panel panel-default panel-lg">
+        <div class="panel-heading">Total Jobs</div>
+        <div class="list-group">
+          <div class="list-group-item">
+            <div class="badge badge-primary">{{overview["jobs-running"]}}</div>Running
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-success">{{overview["jobs-finished"]}}</div>Finished
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-info">{{overview["jobs-cancelled"]}}</div>Canceled
+          </div>
+          <div class="list-group-item">
+            <div class="badge badge-danger">{{overview["jobs-failed"]}}</div>Failed
+          </div>
+        </div>
+      </div>
+    </div>
+  </div>
+  <div class="panel panel-default">
+    <div class="panel-heading">
+      <h3 class="panel-title">Running Jobs</h3>
+    </div>
+    <div class="panel-body">
+      <table class="table table-hover table-clickable">
+        <thead>
+          <tr>
+            <th>Start Time</th>
+            <th>End Time</th>
+            <th>Duration</th>
+            <th>Job Name</th>
+            <th>Job ID</th>
+            <th>Tasks</th>
+            <th>Status</th>
+          </tr>
+        </thead>
+        <tbody>
+          <tr ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job.duration}} ms</td>
+            <td>{{job.name}}</td>
+            <td>{{job.jid}}</td>
+            <td class="label-group">
+              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+            </td>
+            <td> 
+              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+            </td>
+          </tr>
+        </tbody>
+      </table>
+    </div>
+  </div>
+  <div class="panel panel-default">
+    <div class="panel-heading">
+      <h3 class="panel-title">Completed Jobs</h3>
+    </div>
+    <div class="panel-body">
+      <table class="table table-hover table-clickable">
+        <thead>
+          <tr>
+            <th>Start Time</th>
+            <th>End Time</th>
+            <th>Duration</th>
+            <th>Job Name</th>
+            <th>Job ID</th>
+            <th>Tasks</th>
+            <th>Status</th>
+          </tr>
+        </thead>
+        <tbody>
+          <tr ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+            <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
+            <td>{{job.duration}} ms</td>
+            <td>{{job.name}}</td>
+            <td>{{job.jid}}</td>
+            <td class="label-group">
+              <bs-label status="{{status}}" ng-repeat="(status, value) in job.tasks">{{value}}</bs-label>
+            </td>
+            <td> 
+              <bs-label status="{{job.state}}">{{job.state}}</bs-label>
+            </td>
+          </tr>
+        </tbody>
+      </table>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html b/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
new file mode 100644
index 0000000..bf37409
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/partials/taskmanager/index.html
@@ -0,0 +1,57 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-->
+<nav class="navbar navbar-default navbar-fixed-top navbar-main">
+  <div id="fold-button" ng-click="showSidebar()" class="btn btn-default navbar-btn pull-left"><i class="fa fa-navicon"></i></div>
+  <div class="navbar-title">Task Managers</div>
+</nav>
+<div id="content-inner">
+  <table class="table table-clickable table-hover">
+    <thead>
+      <tr>
+        <th>Path, ID</th>
+        <th>Data Port</th>
+        <th>Last Heartbeat</th>
+        <th>All Slots</th>
+        <th>Free Slots</th>
+        <th>CPU Cores</th>
+        <th>Physical Memory</th>
+        <th>Free Memory</th>
+        <th>Flink Managed Memory</th>
+      </tr>
+    </thead>
+    <tbody>
+      <tr ng-repeat="manager in managers" ui-sref="single-manager.metrics({taskmanagerid: manager.id})">
+        <td>
+          {{ manager.path }}
+           
+          <div class="small-label">{{ manager.id }}</div>
+        </td>
+        <td>{{ manager.dataPort }}</td>
+        <td>{{ manager.timeSinceLastHeartbeat | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}</td>
+        <td>{{ manager.slotsNumber }}</td>
+        <td>{{ manager.freeSlots }}</td>
+        <td>{{ manager.cpuCores }}</td>
+        <td>{{ manager.physicalMemory | bytes:MB }}</td>
+        <td>{{ manager.freeMemory | bytes:MB }}</td>
+        <td>{{ manager.managedMemory | bytes:MB }}</td>
+      </tr>
+    </tbody>
+  </table>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
index 8a037ad..4351eb1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -25,7 +25,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.util.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,40 +50,27 @@ class FileSystemBlobStore implements BlobStore {
 	private final String basePath;
 
 	FileSystemBlobStore(Configuration config) throws IOException {
-		StateBackend stateBackend = StateBackend.fromConfig(config);
+		String stateBackendBasePath = config.getString(
+				ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
 
-		if (stateBackend == StateBackend.FILESYSTEM) {
-			String stateBackendBasePath = config.getString(
-					ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
-
-			if (stateBackendBasePath.equals("")) {
-				throw new IllegalConfigurationException(String.format("Missing configuration for " +
-						"file system state backend recovery path. Please specify via " +
-						"'%s' key.", ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH));
-			}
+		if (stateBackendBasePath.equals("")) {
+			throw new IllegalConfigurationException(String.format("Missing configuration for " +
+				"file system state backend recovery path. Please specify via " +
+				"'%s' key.", ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH));
+		}
 
-			stateBackendBasePath += "/blob";
+		stateBackendBasePath += "/blob";
 
-			this.basePath = stateBackendBasePath;
+		this.basePath = stateBackendBasePath;
 
-			try {
-				FileSystem.get(new URI(basePath)).mkdirs(new Path(basePath));
-			}
-			catch (URISyntaxException e) {
-				throw new IOException(e);
-			}
-
-			LOG.info("Created blob directory {}.", basePath);
+		try {
+			FileSystem.get(new URI(basePath)).mkdirs(new Path(basePath));
 		}
-		else {
-			// Nothing else support at the moment
-			throw new IllegalConfigurationException(
-					String.format("Illegal state backend " +
-									"configuration '%s'. Please configure '%s' as state " +
-									"backend and specify the recovery path via '%s' key.",
-							stateBackend, StateBackend.FILESYSTEM,
-							ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH));
+		catch (URISyntaxException e) {
+			throw new IOException(e);
 		}
+
+		LOG.info("Created blob directory {}.", basePath);
 	}
 
 	// - Put ------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
index 62ab440..cb2be64 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
@@ -26,7 +26,7 @@ import org.apache.curator.utils.ZKPaths;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
 import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -92,7 +92,8 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto
 	 * @param client                         The Curator ZooKeeper client
 	 * @param checkpointsPath                The ZooKeeper path for the checkpoints (needs to
 	 *                                       start with a '/')
-	 * @param stateHandleProvider            The state handle provider for checkpoints
+	 * @param stateStorage                   State storage to be used to persist the completed
+	 *                                       checkpoint
 	 * @throws Exception
 	 */
 	public ZooKeeperCompletedCheckpointStore(
@@ -100,16 +101,16 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto
 			ClassLoader userClassLoader,
 			CuratorFramework client,
 			String checkpointsPath,
-			StateHandleProvider<CompletedCheckpoint> stateHandleProvider) throws Exception {
+			StateStorageHelper<CompletedCheckpoint> stateStorage) throws Exception {
 
 		checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint.");
+		checkNotNull(stateStorage, "State storage");
 
 		this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain;
 		this.userClassLoader = checkNotNull(userClassLoader, "User class loader");
 
 		checkNotNull(client, "Curator client");
 		checkNotNull(checkpointsPath, "Checkpoints path");
-		checkNotNull(stateHandleProvider, "State handle provider");
 
 		// Ensure that the checkpoints path exists
 		client.newNamespaceAwareEnsurePath(checkpointsPath)
@@ -118,8 +119,7 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto
 		// All operations will have the path as root
 		this.client = client.usingNamespace(client.getNamespace() + checkpointsPath);
 
-		this.checkpointsInZooKeeper = new ZooKeeperStateHandleStore<>(
-				this.client, stateHandleProvider);
+		this.checkpointsInZooKeeper = new ZooKeeperStateHandleStore<>(this.client, stateStorage);
 
 		this.checkpointStateHandles = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
index 660f8bc..a9ac77a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
@@ -26,7 +26,7 @@ import org.apache.curator.utils.ZKPaths;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
 import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -87,13 +87,21 @@ public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore {
 	/** Flag indicating whether this instance is running. */
 	private boolean isRunning;
 
+	/**
+	 * Submitted job graph store backed by ZooKeeper
+	 *
+	 * @param client ZooKeeper client
+	 * @param currentJobsPath ZooKeeper path for current job graphs
+	 * @param stateStorage State storage used to persist the submitted jobs
+	 * @throws Exception
+	 */
 	public ZooKeeperSubmittedJobGraphStore(
 			CuratorFramework client,
 			String currentJobsPath,
-			StateHandleProvider<SubmittedJobGraph> stateHandleProvider) throws Exception {
+			StateStorageHelper<SubmittedJobGraph> stateStorage) throws Exception {
 
 		checkNotNull(currentJobsPath, "Current jobs path");
-		checkNotNull(stateHandleProvider, "State handle provider");
+		checkNotNull(stateStorage, "State storage");
 
 		// Keep a reference to the original client and not the namespace facade. The namespace
 		// facade cannot be closed.
@@ -104,11 +112,11 @@ public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore {
 				.ensure(client.getZookeeperClient());
 
 		// All operations will have the path as root
-		client = client.usingNamespace(client.getNamespace() + currentJobsPath);
+		CuratorFramework facade = client.usingNamespace(client.getNamespace() + currentJobsPath);
 
-		this.jobGraphsInZooKeeper = new ZooKeeperStateHandleStore<>(client, stateHandleProvider);
+		this.jobGraphsInZooKeeper = new ZooKeeperStateHandleStore<>(facade, stateStorage);
 
-		this.pathCache = new PathChildrenCache(client, "/", false);
+		this.pathCache = new PathChildrenCache(facade, "/", false);
 		pathCache.getListenable().addListener(new SubmittedJobGraphsPathCacheListener());
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractHeapKvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractHeapKvState.java
new file mode 100644
index 0000000..12250b9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractHeapKvState.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Base class for key/value state implementations that are backed by a regular heap hash map. The
+ * concrete implementations define how the state is checkpointed.
+ * 
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ * @param <Backend> The type of the backend that snapshots this key/value state.
+ */
+public abstract class AbstractHeapKvState<K, V, Backend extends StateBackend<Backend>> implements KvState<K, V, Backend> {
+
+	/** Map containing the actual key/value pairs */
+	private final HashMap<K, V> state;
+	
+	/** The serializer for the keys */
+	private final TypeSerializer<K> keySerializer;
+
+	/** The serializer for the values */
+	private final TypeSerializer<V> valueSerializer;
+	
+	/** The value that is returned when no other value has been associated with a key, yet */
+	private final V defaultValue;
+	
+	/** The current key, which the next value methods will refer to */
+	private K currentKey;
+	
+	/**
+	 * Creates a new empty key/value state.
+	 * 
+	 * @param keySerializer The serializer for the keys.
+	 * @param valueSerializer The serializer for the values.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+	 */
+	protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
+									TypeSerializer<V> valueSerializer,
+									V defaultValue) {
+		this(keySerializer, valueSerializer, defaultValue, new HashMap<K, V>());
+	}
+
+	/**
+	 * Creates a new key/value state for the given hash map of key/value pairs.
+	 * 
+	 * @param keySerializer The serializer for the keys.
+	 * @param valueSerializer The serializer for the values.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+	 * @param state The state map to use in this kev/value state. May contain initial state.   
+	 */
+	protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
+									TypeSerializer<V> valueSerializer,
+									V defaultValue,
+									HashMap<K, V> state) {
+		this.state = requireNonNull(state);
+		this.keySerializer = requireNonNull(keySerializer);
+		this.valueSerializer = requireNonNull(valueSerializer);
+		this.defaultValue = defaultValue;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public V value() {
+		V value = state.get(currentKey);
+		return value != null ? value : defaultValue;
+	}
+
+	@Override
+	public void update(V value) {
+		if (value != null) {
+			state.put(currentKey, value);
+		}
+		else {
+			state.remove(currentKey);
+		}
+	}
+
+	@Override
+	public void setCurrentKey(K currentKey) {
+		this.currentKey = currentKey;
+	}
+
+	@Override
+	public int size() {
+		return state.size();
+	}
+
+	@Override
+	public void dispose() {
+		state.clear();
+	}
+
+	/**
+	 * Gets the serializer for the keys.
+	 * @return The serializer for the keys.
+	 */
+	public TypeSerializer<K> getKeySerializer() {
+		return keySerializer;
+	}
+
+	/**
+	 * Gets the serializer for the values.
+	 * @return The serializer for the values.
+	 */
+	public TypeSerializer<V> getValueSerializer() {
+		return valueSerializer;
+	}
+
+	// ------------------------------------------------------------------------
+	//  checkpointing utilities
+	// ------------------------------------------------------------------------
+	
+	protected void writeStateToOutputView(final DataOutputView out) throws IOException {
+		for (Map.Entry<K, V> entry : state.entrySet()) {
+			keySerializer.serialize(entry.getKey(), out);
+			valueSerializer.serialize(entry.getValue(), out);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java
new file mode 100644
index 0000000..5cc16a7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.state.OperatorState;
+
+/**
+ * Key/Value state implementation for user-defined state. The state is backed by a state
+ * backend, which typically follows one of the following patterns: Either the state is stored
+ * in the key/value state object directly (meaning in the executing JVM) and snapshotted by the
+ * state backend into some store (during checkpoints), or the key/value state is in fact backed
+ * by an external key/value store as the state backend, and checkpoints merely record the
+ * metadata of what is considered part of the checkpoint.
+ * 
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public interface KvState<K, V, Backend extends StateBackend<Backend>> extends OperatorState<V> {
+
+	/**
+	 * Sets the current key, which will be used to retrieve values for the next calls to
+	 * {@link #value()} and {@link #update(Object)}.
+	 * 
+	 * @param key The key.
+	 */
+	void setCurrentKey(K key);
+
+	/**
+	 * Creates a snapshot of this state.
+	 * 
+	 * @param checkpointId The ID of the checkpoint for which the snapshot should be created.
+	 * @param timestamp The timestamp of the checkpoint.
+	 * @return A snapshot handle for this key/value state.
+	 * 
+	 * @throws Exception Exceptions during snapshotting the state should be forwarded, so the system
+	 *                   can react to failed snapshots.
+	 */
+	KvStateSnapshot<K, V, Backend> shapshot(long checkpointId, long timestamp) throws Exception;
+
+	/**
+	 * Gets the number of key/value pairs currently stored in the state. Note that is a key
+	 * has been associated with "null", the key is removed from the state an will not
+	 * be counted here.
+	 *
+	 * @return The number of key/value pairs currently stored in the state.
+	 */
+	int size();
+
+	/**
+	 * Disposes the key/value state, releasing all occupied resources.
+	 */
+	void dispose();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvStateSnapshot.java
new file mode 100644
index 0000000..3d6c56c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvStateSnapshot.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+/**
+ * This class represents a snapshot of the {@link KvState}, taken for a checkpoint. Where exactly
+ * the snapshot stores the snapshot data (in this object, in an external data store, etc) depends
+ * on the actual implementation. This snapshot defines merely how to restore the state and
+ * how to discard the state.
+ *
+ * <p>One possible implementation is that this snapshot simply contains a copy of the key/value map.
+ * 
+ * <p>Another possible implementation for this snapshot is that the key/value map is serialized into
+ * a file and this snapshot object contains a pointer to that file.
+ *
+ * @param <K> The type of the key
+ * @param <V> The type of the value
+ * @param <Backend> The type of the backend that can restore the state from this snapshot.
+ */
+public interface KvStateSnapshot<K, V, Backend extends StateBackend<Backend>> extends java.io.Serializable {
+
+	/**
+	 * Loads the key/value state back from this snapshot.
+	 * 
+	 * 
+	 * @param stateBackend The state backend that created this snapshot and can restore the key/value state
+	 *                     from this snapshot.
+	 * @param keySerializer The serializer for the keys.
+	 * @param valueSerializer The serializer for the values.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.   
+	 * @param classLoader The class loader for user-defined types.
+	 * 
+	 * @return An instance of the key/value state loaded from this snapshot.
+	 * 
+	 * @throws Exception Exceptions can occur during the state loading and are forwarded. 
+	 */
+	KvState<K, V, Backend> restoreState(
+			Backend stateBackend,
+			TypeSerializer<K> keySerializer,
+			TypeSerializer<V> valueSerializer,
+			V defaultValue,
+			ClassLoader classLoader) throws Exception;
+
+
+	/**
+	 * Discards the state snapshot, removing any resources occupied by it.
+	 * 
+	 * @throws Exception Exceptions occurring during the state disposal should be forwarded.
+	 */
+	void discardState() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java
new file mode 100644
index 0000000..16ad3fd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java
@@ -0,0 +1,176 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.runtime.util.DataOutputSerializer;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class represents serialized checkpoint data for a collection of elements.
+ */
+public class SerializedCheckpointData implements java.io.Serializable {
+
+	private static final long serialVersionUID = -8783744683896503488L;
+	
+	/** ID of the checkpoint for which the IDs are stored */
+	private final long checkpointId;
+
+	/** The serialized elements */
+	private final byte[] serializedData;
+
+	/** The number of elements in the checkpoint */
+	private final int numIds;
+
+	/**
+	 * Creates a SerializedCheckpointData object for the given serialized data.
+	 * 
+	 * @param checkpointId The checkpointId of the checkpoint.
+	 * @param serializedData The serialized IDs in this checkpoint.
+	 * @param numIds The number of IDs in the checkpoint.
+	 */
+	public SerializedCheckpointData(long checkpointId, byte[] serializedData, int numIds) {
+		this.checkpointId = checkpointId;
+		this.serializedData = serializedData;
+		this.numIds = numIds;
+	}
+
+	/**
+	 * Gets the checkpointId of the checkpoint.
+	 * @return The checkpointId of the checkpoint.
+	 */
+	public long getCheckpointId() {
+		return checkpointId;
+	}
+
+	/**
+	 * Gets the binary data for the serialized elements.
+	 * @return The binary data for the serialized elements.
+	 */
+	public byte[] getSerializedData() {
+		return serializedData;
+	}
+
+	/**
+	 * Gets the number of IDs in the checkpoint.
+	 * @return The number of IDs in the checkpoint.
+	 */
+	public int getNumIds() {
+		return numIds;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Serialize to Checkpoint
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Converts a list of checkpoints with elements into an array of SerializedCheckpointData.
+	 * 
+	 * @param checkpoints The checkpoints to be converted into IdsCheckpointData.
+	 * @param serializer The serializer to serialize the IDs.
+	 * @param <T> The type of the ID.
+	 * @return An array of serializable SerializedCheckpointData, one per entry in the 
+	 * 
+	 * @throws IOException Thrown, if the serialization fails.
+	 */
+	public static <T> SerializedCheckpointData[] fromDeque(ArrayDeque<Tuple2<Long, List<T>>> checkpoints,
+												TypeSerializer<T> serializer) throws IOException {
+		return fromDeque(checkpoints, serializer, new DataOutputSerializer(128));
+	}
+
+	/**
+	 * Converts a list of checkpoints into an array of SerializedCheckpointData.
+	 *
+	 * @param checkpoints The checkpoints to be converted into IdsCheckpointData.
+	 * @param serializer The serializer to serialize the IDs.
+	 * @param outputBuffer The reusable serialization buffer.
+	 * @param <T> The type of the ID.
+	 * @return An array of serializable SerializedCheckpointData, one per entry in the 
+	 *
+	 * @throws IOException Thrown, if the serialization fails.
+	 */
+	public static <T> SerializedCheckpointData[] fromDeque(ArrayDeque<Tuple2<Long, List<T>>> checkpoints,
+												TypeSerializer<T> serializer,
+												DataOutputSerializer outputBuffer) throws IOException {
+		SerializedCheckpointData[] serializedCheckpoints = new SerializedCheckpointData[checkpoints.size()];
+		
+		int pos = 0;
+		for (Tuple2<Long, List<T>> checkpoint : checkpoints) {
+			outputBuffer.clear();
+			List<T> checkpointIds = checkpoint.f1;
+			
+			for (T id : checkpointIds) {
+				serializer.serialize(id, outputBuffer);
+			}
+
+			serializedCheckpoints[pos++] = new SerializedCheckpointData(
+					checkpoint.f0, outputBuffer.getCopyOfBuffer(), checkpointIds.size());
+		}
+		
+		return serializedCheckpoints;
+	}
+
+	// ------------------------------------------------------------------------
+	//  De-Serialize from Checkpoint
+	// ------------------------------------------------------------------------
+
+	/**
+	 * De-serializes an array of SerializedCheckpointData back into an ArrayDeque of element checkpoints.
+	 * 
+	 * @param data The data to be deserialized.
+	 * @param serializer The serializer used to deserialize the data.
+	 * @param <T> The type of the elements.
+	 * @return An ArrayDeque of element checkpoints.
+	 * 
+	 * @throws IOException Thrown, if the serialization fails.
+	 */
+	public static <T> ArrayDeque<Tuple2<Long, List<T>>> toDeque(
+			SerializedCheckpointData[] data, TypeSerializer<T> serializer) throws IOException
+	{
+		ArrayDeque<Tuple2<Long, List<T>>> deque = new ArrayDeque<>(data.length);
+		DataInputDeserializer deser = null;
+		
+		for (SerializedCheckpointData checkpoint : data) {
+			byte[] serializedData = checkpoint.getSerializedData();
+			if (deser == null) {
+				deser = new DataInputDeserializer(serializedData, 0, serializedData.length);
+			}
+			else {
+				deser.setBuffer(serializedData, 0, serializedData.length);
+			}
+			
+			final List<T> ids = new ArrayList<>(checkpoint.getNumIds());
+			final int numIds = checkpoint.getNumIds();
+			
+			for (int i = 0; i < numIds; i++) {
+				ids.add(serializer.deserialize(deser));
+			}
+
+			deque.addLast(new Tuple2<Long, List<T>>(checkpoint.checkpointId, ids));
+		}
+		
+		return deque;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
index 7aa1ccf..f8b1cfd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
@@ -18,22 +18,196 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 
-public enum StateBackend {
-	JOBMANAGER, FILESYSTEM;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * A state backend defines how state is stored and snapshotted during checkpoints.
+ * 
+ * @param <Backend> The type of backend itself. This generic parameter is used to refer to the
+ *                  type of backend when creating state backed by this backend.
+ */
+public abstract class StateBackend<Backend extends StateBackend<Backend>> implements java.io.Serializable {
+	
+	private static final long serialVersionUID = 4620413814639220247L;
+	
+	// ------------------------------------------------------------------------
+	//  initialization and cleanup
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * This method is called by the task upon deployment to initialize the state backend for
+	 * data for a specific job.
+	 * 
+	 * @param job The ID of the job for which the state backend instance checkpoints data.
+	 * @throws Exception Overwritten versions of this method may throw exceptions, in which
+	 *                   case the job that uses the state backend is considered failed during
+	 *                   deployment.
+	 */
+	public abstract void initializeForJob(JobID job) throws Exception;
+
+	/**
+	 * Disposes all state associated with the current job.
+	 * 
+	 * @throws Exception Exceptions may occur during disposal of the state and should be forwarded.
+	 */
+	public abstract void disposeAllStateForCurrentJob() throws Exception;
+
+	/**
+	 * Closes the state backend, releasing all internal resources, but does not delete any persistent
+	 * checkpoint data.
+	 * 
+	 * @throws Exception Exceptions can be forwarded and will be logged by the system
+	 */
+	public abstract void close() throws Exception;
+	
+	// ------------------------------------------------------------------------
+	//  key/value state
+	// ------------------------------------------------------------------------
 
 	/**
-	 * Returns the configured {@link StateBackend}.
+	 * Creates a key/value state backed by this state backend.
+	 * 
+	 * @param keySerializer The serializer for the key.
+	 * @param valueSerializer The serializer for the value.
+	 * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+	 * @param <K> The type of the key.
+	 * @param <V> The type of the value.
+	 * 
+	 * @return A new key/value state backed by this backend.
+	 * 
+	 * @throws Exception Exceptions may occur during initialization of the state and should be forwarded.
+	 */
+	public abstract <K, V> KvState<K, V, Backend> createKvState(
+			TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+			V defaultValue) throws Exception;
+	
+	
+	// ------------------------------------------------------------------------
+	//  storing state for a checkpoint
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates an output stream that writes into the state of the given checkpoint. When the stream
+	 * is closes, it returns a state handle that can retrieve the state back.
+	 * 
+	 * @param checkpointID The ID of the checkpoint.
+	 * @param timestamp The timestamp of the checkpoint.
+	 * @return An output stream that writes state for the given checkpoint.
+	 * 
+	 * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
+	 */
+	public abstract CheckpointStateOutputStream createCheckpointStateOutputStream(
+			long checkpointID, long timestamp) throws Exception;
+	
+	/**
+	 * Creates a {@link DataOutputView} stream that writes into the state of the given checkpoint.
+	 * When the stream is closes, it returns a state handle that can retrieve the state back.
 	 *
-	 * @param config The config to parse
-	 * @return Configured state backend or {@link ConfigConstants#DEFAULT_RECOVERY_MODE} if not
-	 * configured.
-	 */
-	public static StateBackend fromConfig(Configuration config) {
-		return StateBackend.valueOf(config.getString(
-				ConfigConstants.STATE_BACKEND,
-				ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase());
+	 * @param checkpointID The ID of the checkpoint.
+	 * @param timestamp The timestamp of the checkpoint.
+	 * @return An DataOutputView stream that writes state for the given checkpoint.
+	 *
+	 * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
+	 */
+	public CheckpointStateOutputView createCheckpointStateOutputView(
+			long checkpointID, long timestamp) throws Exception {
+		return new CheckpointStateOutputView(createCheckpointStateOutputStream(checkpointID, timestamp));
+	}
+
+	/**
+	 * Writes the given state into the checkpoint, and returns a handle that can retrieve the state back.
+	 * 
+	 * @param state The state to be checkpointed.
+	 * @param checkpointID The ID of the checkpoint.
+	 * @param timestamp The timestamp of the checkpoint.
+	 * @param <S> The type of the state.
+	 * 
+	 * @return A state handle that can retrieve the checkpoined state.
+	 * 
+	 * @throws Exception Exceptions may occur during serialization / storing the state and should be forwarded.
+	 */
+	public abstract <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+			S state, long checkpointID, long timestamp) throws Exception;
+	
+	
+	// ------------------------------------------------------------------------
+	//  Checkpoint state output stream
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A dedicated output stream that produces a {@link StreamStateHandle} when closed.
+	 */
+	public static abstract class CheckpointStateOutputStream extends OutputStream {
+
+		/**
+		 * Closes the stream and gets a state handle that can create an input stream
+		 * producing the data written to this stream.
+		 * 
+		 * @return A state handle that can create an input stream producing the data written to this stream.
+		 * @throws IOException Thrown, if the stream cannot be closed.
+		 */
+		public abstract StreamStateHandle closeAndGetHandle() throws IOException;
+	}
+
+	/**
+	 * A dedicated DataOutputView stream that produces a {@code StateHandle<DataInputView>} when closed.
+	 */
+	public static final class CheckpointStateOutputView extends DataOutputViewStreamWrapper {
+		
+		private final CheckpointStateOutputStream out;
+		
+		public CheckpointStateOutputView(CheckpointStateOutputStream out) {
+			super(out);
+			this.out = out;
+		}
+
+		/**
+		 * Closes the stream and gets a state handle that can create a DataInputView.
+		 * producing the data written to this stream.
+		 *
+		 * @return A state handle that can create an input stream producing the data written to this stream.
+		 * @throws IOException Thrown, if the stream cannot be closed.
+		 */
+		public StateHandle<DataInputView> closeAndGetHandle() throws IOException {
+			return new DataInputViewHandle(out.closeAndGetHandle());
+		}
+
+		@Override
+		public void close() throws IOException {
+			out.close();
+		}
+	}
+
+	/**
+	 * Simple state handle that resolved a {@link DataInputView} from a StreamStateHandle.
+	 */
+	private static final class DataInputViewHandle implements StateHandle<DataInputView> {
+
+		private static final long serialVersionUID = 2891559813513532079L;
+		
+		private final StreamStateHandle stream;
+
+		private DataInputViewHandle(StreamStateHandle stream) {
+			this.stream = stream;
+		}
+
+		@Override
+		public DataInputView getState(ClassLoader userCodeClassLoader) throws Exception {
+			return new DataInputViewStreamWrapper(stream.getState(userCodeClassLoader)); 
+		}
+
+		@Override
+		public void discardState() throws Exception {
+			stream.discardState();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
new file mode 100644
index 0000000..5b622eb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * A factory to create a specific state backend. The state backend creation gets a Configuration
+ * object that can be used to read further config values.
+ * 
+ * @param <T> The type of the state backend created.
+ */
+public interface StateBackendFactory<T extends StateBackend<T>> {
+
+	/**
+	 * Creates the state backend, optionally using the given configuration.
+	 * 
+	 * @param config The Flink configuration (loaded by the TaskManager).
+	 * @return The created state backend. 
+	 * 
+	 * @throws Exception Exceptions during instantiation can be forwarded.
+	 */
+	StateBackend<T> createFromConfig(Configuration config) throws Exception;
+}


[08/47] flink git commit: [FLINK-2652] [tests] Temporary ignore flakey PartitionRequestClientFactoryTest

Posted by tr...@apache.org.
[FLINK-2652] [tests] Temporary ignore flakey PartitionRequestClientFactoryTest


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

Branch: refs/heads/master
Commit: b23394640c8b4f8751c16f7c7dee5bc33de91bbd
Parents: 5671c77
Author: Ufuk Celebi <uc...@apache.org>
Authored: Sun Sep 20 13:08:24 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:51 2015 +0200

----------------------------------------------------------------------
 .../io/network/netty/PartitionRequestClientFactoryTest.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b2339464/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
index 24a2a5c..56e5bde 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
@@ -25,6 +25,8 @@ import io.netty.channel.ChannelPromise;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.net.NetUtils;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -41,9 +43,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+@Ignore
 public class PartitionRequestClientFactoryTest {
 
-	private final static int SERVER_PORT = 10021;
+	private final static int SERVER_PORT = NetUtils.getAvailablePort();
 
 	@Test
 	public void testResourceReleaseAfterInterruptedConnect() throws Exception {


[21/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf
new file mode 100644
index 0000000..ed9372f
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.ttf differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff
new file mode 100644
index 0000000..8b280b9
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2 b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2
new file mode 100644
index 0000000..3311d58
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.woff2 differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/images/flink-logo.png
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/images/flink-logo.png b/flink-runtime-web/src/main/resources/web/images/flink-logo.png
new file mode 100644
index 0000000..088fb27
Binary files /dev/null and b/flink-runtime-web/src/main/resources/web/images/flink-logo.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/index.html b/flink-runtime-web/src/main/resources/web/index.html
new file mode 100644
index 0000000..a6df249
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/index.html
@@ -0,0 +1,55 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--><!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Apache Flink Web Dashboard</title>
+    <link rel="stylesheet" href="css/vendor.css" type="text/css">
+    <link rel="stylesheet" href="css/index.css" type="text/css">
+    <script src="js/vendor.js"></script>
+    <script src="js/index.js"></script>
+  </head>
+  <body ng-app="flinkApp" ng-strict-di>
+    <div id="sidebar" ng-class="{ 'sidebar-visible': sidebarVisible }">
+      <nav class="navbar navbar-inverse navbar-static-top">
+        <div class="navbar-header"><a ui-sref="overview" class="navbar-brand"><img alt="Apache Flink Dashboard" src="images/flink-logo.png" class="logo"></a><a ui-sref="overview" class="navbar-brand navbar-brand-text">Apache Flink Dashboard</a></div>
+      </nav>
+      <div class="navbar navbar-sidebar">
+        <ul class="nav">
+          <li><a ui-sref="overview" ui-sref-active="active"><i class="fa fa-dashboard fa-fw"></i> 
+              Overview</a></li>
+          <li><a ui-sref="running-jobs" ui-sref-active="active"><i class="fa fa-tasks fa-fw"></i> 
+              Running Jobs</a></li>
+          <li><a ui-sref="completed-jobs" ui-sref-active="active"><i class="fa fa-check-circle fa-fw"></i> 
+              Completed Jobs</a></li>
+          <li><a ui-sref="all-manager" ui-sref-active="active"><i class="fa fa-sitemap fa-fw"></i> 
+              Task Managers</a></li>
+          <li><a ui-sref="jobmanager.config" ui-sref-active="active"><i class="fa fa-server fa-fw"></i> 
+              Job Manager</a></li>
+        </ul>
+      </div>
+    </div>
+    <div id="content" ng-class="{ 'sidebar-visible': sidebarVisible }">
+      <div ui-view="main"></div>
+    </div>
+  </body>
+</html>
\ No newline at end of file


[44/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
deleted file mode 100644
index 1ee89d4..0000000
--- a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
+++ /dev/null
@@ -1,565 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1">
-<metadata></metadata>
-<defs>
-<font id="fontawesomeregular" horiz-adv-x="1536" >
-<font-face units-per-em="1792" ascent="1536" descent="-256" />
-<missing-glyph horiz-adv-x="448" />
-<glyph unicode=" "  horiz-adv-x="448" />
-<glyph unicode="&#x09;" horiz-adv-x="448" />
-<glyph unicode="&#xa0;" horiz-adv-x="448" />
-<glyph unicode="&#xa8;" horiz-adv-x="1792" />
-<glyph unicode="&#xa9;" horiz-adv-x="1792" />
-<glyph unicode="&#xae;" horiz-adv-x="1792" />
-<glyph unicode="&#xb4;" horiz-adv-x="1792" />
-<glyph unicode="&#xc6;" horiz-adv-x="1792" />
-<glyph unicode="&#xd8;" horiz-adv-x="1792" />
-<glyph unicode="&#x2000;" horiz-adv-x="768" />
-<glyph unicode="&#x2001;" horiz-adv-x="1537" />
-<glyph unicode="&#x2002;" horiz-adv-x="768" />
-<glyph unicode="&#x2003;" horiz-adv-x="1537" />
-<glyph unicode="&#x2004;" horiz-adv-x="512" />
-<glyph unicode="&#x2005;" horiz-adv-x="384" />
-<glyph unicode="&#x2006;" horiz-adv-x="256" />
-<glyph unicode="&#x2007;" horiz-adv-x="256" />
-<glyph unicode="&#x2008;" horiz-adv-x="192" />
-<glyph unicode="&#x2009;" horiz-adv-x="307" />
-<glyph unicode="&#x200a;" horiz-adv-x="85" />
-<glyph unicode="&#x202f;" horiz-adv-x="307" />
-<glyph unicode="&#x205f;" horiz-adv-x="384" />
-<glyph unicode="&#x2122;" horiz-adv-x="1792" />
-<glyph unicode="&#x221e;" horiz-adv-x="1792" />
-<glyph unicode="&#x2260;" horiz-adv-x="1792" />
-<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
-<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
-<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
-<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
-<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
-<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
-<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
-<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
-<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
-<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
-<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
-<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
-<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
-<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
-<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
-<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
-<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
-<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
-<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
-<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
-<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
-<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
-<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
-<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
-<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
-<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
-<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
-<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
-<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
-<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
-<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
-<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
-<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
-<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
-<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
-<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
-<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
-<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
-<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
-<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
-<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
-<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
-<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
-<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
-<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
-<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
-<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
-<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
-<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
-<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
-<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
-<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
-<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
-<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
-<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
-<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
-<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
-<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
-<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
-<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
-<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
-<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
-<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf082;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 q0 119 84.5 203.5t203.5 84.5h960z" />
-<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
-<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
-<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
-<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
-<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
-<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
-<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
-<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
-<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
-<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
-<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
-<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
-<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
-<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
-<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
-<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
-<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
-<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
-<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
-<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
-<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
-<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
-<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5

<TRUNCATED>

[07/47] flink git commit: [FLINK-2792] [jobmanager, logging] Set actor message log level to TRACE

Posted by tr...@apache.org.
[FLINK-2792] [jobmanager, logging] Set actor message log level to TRACE


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

Branch: refs/heads/master
Commit: 3aaee1e51bae383b8c884a99b5523c25310c011d
Parents: b233946
Author: Ufuk Celebi <uc...@apache.org>
Authored: Wed Sep 30 16:38:37 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:51 2015 +0200

----------------------------------------------------------------------
 .../src/main/scala/org/apache/flink/runtime/LogMessages.scala  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3aaee1e5/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala
index e1e8961..e4c1747 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala
@@ -29,18 +29,18 @@ trait LogMessages extends FlinkActor {
       override def isDefinedAt(x: Any): Boolean = _receive.isDefinedAt(x)
 
       override def apply(x: Any): Unit = {
-        if (!log.isDebugEnabled) {
+        if (!log.isTraceEnabled) {
           _receive(x)
         }
         else {
-          log.debug(s"Received message $x at ${context.self.path} from ${context.sender()}.")
+          log.trace(s"Received message $x at ${context.self.path} from ${context.sender()}.")
 
           val start = System.nanoTime()
 
           _receive(x)
 
           val duration = (System.nanoTime() - start) / 1000000
-          log.debug(s"Handled message $x in $duration ms from ${context.sender()}.")
+          log.trace(s"Handled message $x in $duration ms from ${context.sender()}.")
         }
       }
     }


[36/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf
new file mode 100644
index 0000000..ed9372f
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.ttf differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff
new file mode 100644
index 0000000..8b280b9
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2 b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2
new file mode 100644
index 0000000..3311d58
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/fonts/fontawesome-webfont.woff2 differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/images/flink-logo.png
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/images/flink-logo.png b/flink-runtime-web/web-dashboard/web/images/flink-logo.png
new file mode 100644
index 0000000..088fb27
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/images/flink-logo.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/index.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/index.html b/flink-runtime-web/web-dashboard/web/index.html
new file mode 100644
index 0000000..a6df249
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/index.html
@@ -0,0 +1,55 @@
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+--><!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Apache Flink Web Dashboard</title>
+    <link rel="stylesheet" href="css/vendor.css" type="text/css">
+    <link rel="stylesheet" href="css/index.css" type="text/css">
+    <script src="js/vendor.js"></script>
+    <script src="js/index.js"></script>
+  </head>
+  <body ng-app="flinkApp" ng-strict-di>
+    <div id="sidebar" ng-class="{ 'sidebar-visible': sidebarVisible }">
+      <nav class="navbar navbar-inverse navbar-static-top">
+        <div class="navbar-header"><a ui-sref="overview" class="navbar-brand"><img alt="Apache Flink Dashboard" src="images/flink-logo.png" class="logo"></a><a ui-sref="overview" class="navbar-brand navbar-brand-text">Apache Flink Dashboard</a></div>
+      </nav>
+      <div class="navbar navbar-sidebar">
+        <ul class="nav">
+          <li><a ui-sref="overview" ui-sref-active="active"><i class="fa fa-dashboard fa-fw"></i> 
+              Overview</a></li>
+          <li><a ui-sref="running-jobs" ui-sref-active="active"><i class="fa fa-tasks fa-fw"></i> 
+              Running Jobs</a></li>
+          <li><a ui-sref="completed-jobs" ui-sref-active="active"><i class="fa fa-check-circle fa-fw"></i> 
+              Completed Jobs</a></li>
+          <li><a ui-sref="all-manager" ui-sref-active="active"><i class="fa fa-sitemap fa-fw"></i> 
+              Task Managers</a></li>
+          <li><a ui-sref="jobmanager.config" ui-sref-active="active"><i class="fa fa-server fa-fw"></i> 
+              Job Manager</a></li>
+        </ul>
+      </div>
+    </div>
+    <div id="content" ng-class="{ 'sidebar-visible': sidebarVisible }">
+      <div ui-view="main"></div>
+    </div>
+  </body>
+</html>
\ No newline at end of file


[41/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/js/index.js.orig
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/index.js.orig b/flink-runtime-web/src/main/resources/web/js/index.js.orig
deleted file mode 100644
index 018e743..0000000
--- a/flink-runtime-web/src/main/resources/web/js/index.js.orig
+++ /dev/null
@@ -1,1150 +0,0 @@
-angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
-  $rootScope.sidebarVisible = false;
-  return $rootScope.showSidebar = function() {
-    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
-    return $rootScope.sidebarClass = 'force-show';
-  };
-}]).value('flinkConfig', {
-  "refresh-interval": 10000
-}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
-  return MainService.loadConfig().then(function(config) {
-    angular.extend(flinkConfig, config);
-    JobsService.listJobs();
-    return $interval(function() {
-      return JobsService.listJobs();
-    }, flinkConfig["refresh-interval"]);
-  });
-}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
-  return $uiViewScrollProvider.useAnchorScroll();
-}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
-  $stateProvider.state("overview", {
-    url: "/overview",
-    views: {
-      main: {
-        templateUrl: "partials/overview.html",
-        controller: 'OverviewController'
-      }
-    }
-  }).state("running-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: true,
-    views: {
-      main: {
-        templateUrl: "partials/jobs/job.html",
-        controller: 'SingleJobController'
-      }
-    }
-  }).state("single-job.plan", {
-    url: "",
-    abstract: true,
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.plan.html",
-        controller: 'JobPlanController'
-      }
-    }
-  }).state("single-job.plan.overview", {
-    url: "",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
-        controller: 'JobPlanOverviewController'
-      }
-    }
-  }).state("single-job.plan.accumulators", {
-    url: "/accumulators",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
-        controller: 'JobPlanAccumulatorsController'
-      }
-    }
-  }).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.statistics", {
-    url: "/statistics",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.statistics.html"
-      }
-    }
-  }).state("single-job.exceptions", {
-    url: "/exceptions",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.exceptions.html",
-        controller: 'JobExceptionsController'
-      }
-    }
-  }).state("single-job.properties", {
-    url: "/properties",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.properties.html",
-        controller: 'JobPropertiesController'
-      }
-    }
-  }).state("single-job.config", {
-    url: "/config",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.config.html"
-      }
-    }
-  }).state("taskmanagers", {
-    url: "/taskmanagers",
-    views: {
-      main: {
-        templateUrl: "partials/taskmanagers/index.html",
-        controller: 'TaskManagersController'
-      }
-    }
-  }).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"
-      }
-    }
-  }).state("jobmanager.logfile", {
-    url: "/logfile",
-    views: {
-      details: {
-        templateUrl: "partials/jobmanager/logfile.html"
-      }
-    }
-  });
-  return $urlRouterProvider.otherwise("/overview");
-}]);
-
-angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
-  return {
-    transclude: true,
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: "@"
-    },
-    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'label label-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
-  return {
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: '@'
-    },
-    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('tableProperty', function() {
-  return {
-    replace: true,
-    scope: {
-      value: '='
-    },
-    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
-  };
-});
-
-angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
-  var amDurationFormatExtendedFilter;
-  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
-    if (typeof value === "undefined" || value === null) {
-      return "";
-    }
-    return moment.duration(value, format).format(durationFormat, {
-      trim: false
-    });
-  };
-  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
-  return amDurationFormatExtendedFilter;
-}]).filter("humanizeText", function() {
-  return function(text) {
-    if (text) {
-      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
-    } else {
-      return '';
-    }
-  };
-}).filter("bytes", function() {
-  return function(bytes, precision) {
-    var number, units;
-    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
-      return "-";
-    }
-    if (typeof precision === "undefined") {
-      precision = 1;
-    }
-    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
-    number = Math.floor(Math.log(bytes) / Math.log(1024));
-    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
-  };
-});
-
-angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("/config").success(function(data, status, headers, config) {
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
-  return JobManagerConfigService.loadConfig().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['config'] = data;
-  });
-}]);
-
-angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var config;
-  config = {};
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("/jobmanager/config").success(function(data, status, headers, config) {
-      config = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('running');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
-  var refresher;
-  console.log('SingleJobController');
-  $scope.jobid = $stateParams.jobid;
-  $scope.job = null;
-  $scope.plan = null;
-  $scope.vertices = null;
-  JobsService.loadJob($stateParams.jobid).then(function(data) {
-    $scope.job = data;
-    $scope.plan = data.plan;
-    return $scope.vertices = data.vertices;
-  });
-  refresher = $interval(function() {
-    return JobsService.loadJob($stateParams.jobid).then(function(data) {
-      $scope.job = data;
-      return $scope.$broadcast('reload');
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    $scope.job = null;
-    $scope.plan = null;
-    $scope.vertices = null;
-    return $interval.cancel(refresher);
-  });
-}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobPlanController');
-  $scope.nodeid = null;
-  $scope.nodeUnfolded = false;
-  $scope.stateList = JobsService.stateList();
-  $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      $scope.accumulators = null;
-      return $scope.$broadcast('reload');
-    } else {
-      $scope.nodeid = null;
-      $scope.nodeUnfolded = false;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      return $scope.accumulators = null;
-    }
-  };
-  $scope.deactivateNode = function() {
-    $scope.nodeid = null;
-    $scope.nodeUnfolded = false;
-    $scope.vertex = null;
-    $scope.subtasks = null;
-    return $scope.accumulators = null;
-  };
-  return $scope.toggleFold = function() {
-    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
-  };
-}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanOverviewController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
-    JobsService.getSubtasks($scope.nodeid).then(function(data) {
-      return $scope.subtasks = data;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanOverviewController');
-    if ($scope.nodeid) {
-      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
-        return $scope.subtasks = data;
-      });
-    }
-  });
-}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanAccumulatorsController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
-    JobsService.getAccumulators($scope.nodeid).then(function(data) {
-      $scope.accumulators = data.main;
-      return $scope.subtaskAccumulators = data.subtasks;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanAccumulatorsController');
-    if ($scope.nodeid) {
-      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
-        $scope.accumulators = data.main;
-        return $scope.subtaskAccumulators = data.subtasks;
-      });
-    }
-  });
-}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobTimelineVertexController');
-  JobsService.getVertex($stateParams.vertexId).then(function(data) {
-    return $scope.vertex = data;
-  });
-  return $scope.$on('reload', function(event) {
-    console.log('JobTimelineVertexController');
-    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
-      return $scope.vertex = data;
-    });
-  });
-}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  return JobsService.loadExceptions().then(function(data) {
-    return $scope.exceptions = data;
-  });
-}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPropertiesController');
-  return $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      return JobsService.getNode(nodeid).then(function(data) {
-        return $scope.node = data;
-      });
-    } else {
-      $scope.nodeid = null;
-      return $scope.node = null;
-    }
-  };
-}]);
-
-angular.module('flinkApp').directive('vertex', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
-    scope: {
-      data: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data.subtasks, function(subtask, i) {
-          var times;
-          times = [
-            {
-              label: "Scheduled",
-              color: "#666",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["SCHEDULED"],
-              ending_time: subtask.timestamps["DEPLOYING"],
-              type: 'regular'
-            }, {
-              label: "Deploying",
-              color: "#aaa",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["DEPLOYING"],
-              ending_time: subtask.timestamps["RUNNING"],
-              type: 'regular'
-            }
-          ];
-          if (subtask.timestamps["FINISHED"] > 0) {
-            times.push({
-              label: "Running",
-              color: "#ddd",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["RUNNING"],
-              ending_time: subtask.timestamps["FINISHED"],
-              type: 'regular'
-            });
-          }
-          return testData.push({
-            label: "(" + subtask.subtask + ") " + subtask.host,
-            times: times
-          });
-        });
-        chart = d3.timeline().stack().tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("single").labelFormat(function(label) {
-          return label;
-        }).margin({
-          left: 100,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).relativeTime();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      analyzeTime(scope.data);
-    }
-  };
-}]).directive('timeline', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline' width='0' height='0'></svg>",
-    scope: {
-      vertices: "=",
-      jobid: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl, translateLabel;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      translateLabel = function(label) {
-        return label.replace("&gt;", ">");
-      };
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data, function(vertex) {
-          if (vertex['start-time'] > -1) {
-            if (vertex.type === 'scheduled') {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#cccccc",
-                    borderColor: "#555555",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    type: vertex.type
-                  }
-                ]
-              });
-            } else {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#d9f1f7",
-                    borderColor: "#62cdea",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    link: vertex.id,
-                    type: vertex.type
-                  }
-                ]
-              });
-            }
-          }
-        });
-        chart = d3.timeline().stack().click(function(d, i, datum) {
-          if (d.link) {
-            return $state.go("single-job.timeline.vertex", {
-              jobid: scope.jobid,
-              vertexId: d.link
-            });
-          }
-        }).tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("main").margin({
-          left: 0,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).showBorderLine().showHourTimeline();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      scope.$watch(attrs.vertices, function(data) {
-        if (data) {
-          return analyzeTime(data);
-        }
-      });
-    }
-  };
-}]).directive('jobPlan', ["$timeout", function($timeout) {
-  return {
-    template: "<svg class='graph' width='500' height='400'><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: '=',
-      setNode: '&'
-    },
-    link: function(scope, elem, attrs) {
-      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
-      g = null;
-      mainZoom = d3.behavior.zoom();
-      subgraphs = [];
-      jobid = attrs.jobid;
-      mainSvgElement = elem.children()[0];
-      mainG = elem.children().children()[0];
-      mainTmpElement = elem.children()[1];
-      d3mainSvg = d3.select(mainSvgElement);
-      d3mainSvgG = d3.select(mainG);
-      d3tmpSvg = d3.select(mainTmpElement);
-      containerW = elem.width();
-      angular.element(elem.children()[0]).width(containerW);
-      scope.zoomIn = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() < 2.99) {
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          mainZoom.scale(mainZoom.scale() + 0.1);
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      scope.zoomOut = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() > 0.31) {
-          mainZoom.scale(mainZoom.scale() - 0.1);
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      createLabelEdge = function(el) {
-        var labelValue;
-        labelValue = "";
-        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
-          labelValue += "<div class='edge-label'>";
-          if (el.ship_strategy != null) {
-            labelValue += el.ship_strategy;
-          }
-          if (el.temp_mode !== undefined) {
-            labelValue += " (" + el.temp_mode + ")";
-          }
-          if (el.local_strategy !== undefined) {
-            labelValue += ",<br>" + el.local_strategy;
-          }
-          labelValue += "</div>";
-        }
-        return labelValue;
-      };
-      isSpecialIterationNode = function(info) {
-        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
-      };
-      getNodeType = function(el, info) {
-        if (info === "mirror") {
-          return 'node-mirror';
-        } else if (isSpecialIterationNode(info)) {
-          return 'node-iteration';
-        } else {
-          return 'node-normal';
-        }
-      };
-      createLabelNode = function(el, info, maxW, maxH) {
-        var labelValue, stepName;
-        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
-        if (info === "mirror") {
-          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
-        } else {
-          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
-        }
-        if (el.description === "") {
-          labelValue += "";
-        } else {
-          stepName = el.description;
-          stepName = shortenString(stepName);
-          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
-        }
-        if (el.step_function != null) {
-          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
-        } else {
-          if (isSpecialIterationNode(info)) {
-            labelValue += "<h5>" + info + " Node</h5>";
-          }
-          if (el.parallelism !== "") {
-            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
-          }
-          if (el.operator !== undefined) {
-            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
-          }
-        }
-        labelValue += "</div>";
-        return labelValue;
-      };
-      extendLabelNodeForIteration = function(id, maxW, maxH) {
-        var labelValue, svgID;
-        svgID = "svg-" + id;
-        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
-        return labelValue;
-      };
-      shortenString = function(s) {
-        var sbr;
-        if (s.charAt(0) === "<") {
-          s = s.replace("<", "&lt;");
-          s = s.replace(">", "&gt;");
-        }
-        sbr = "";
-        while (s.length > 30) {
-          sbr = sbr + s.substring(0, 30) + "<br>";
-          s = s.substring(30, s.length);
-        }
-        sbr = sbr + s;
-        return sbr;
-      };
-      createNode = function(g, data, el, isParent, maxW, maxH) {
-        if (isParent == null) {
-          isParent = false;
-        }
-        if (el.id === data.partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "partialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "partialSolution")
-          });
-        } else if (el.id === data.next_partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextPartialSolution")
-          });
-        } else if (el.id === data.workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "workset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "workset")
-          });
-        } else if (el.id === data.next_workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextWorkset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextWorkset")
-          });
-        } else if (el.id === data.solution_set) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionSet", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionSet")
-          });
-        } else if (el.id === data.solution_delta) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionDelta", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionDelta")
-          });
-        } else {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "")
-          });
-        }
-      };
-      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
-        var missingNode;
-        if (existingNodes.indexOf(pred.id) !== -1) {
-          return g.setEdge(pred.id, el.id, {
-            label: createLabelEdge(pred),
-            labelType: 'html',
-            arrowhead: 'normal'
-          });
-        } else {
-          missingNode = searchForNode(data, pred.id);
-          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
-            missingNodes.push(missingNode.id);
-            g.setNode(missingNode.id, {
-              label: createLabelNode(missingNode, "mirror"),
-              labelType: 'html',
-              "class": getNodeType(missingNode, 'mirror')
-            });
-            return g.setEdge(missingNode.id, el.id, {
-              label: createLabelEdge(missingNode),
-              labelType: 'html'
-            });
-          }
-        }
-      };
-      loadJsonToDagre = function(g, data) {
-        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
-        existingNodes = [];
-        missingNodes = [];
-        if (data.nodes != null) {
-          toIterate = data.nodes;
-        } else {
-          toIterate = data.step_function;
-          isParent = true;
-        }
-        for (k = 0, len = toIterate.length; k < len; k++) {
-          el = toIterate[k];
-          maxW = 0;
-          maxH = 0;
-          if (el.step_function) {
-            sg = new dagreD3.graphlib.Graph({
-              multigraph: true,
-              compound: true
-            }).setGraph({
-              nodesep: 20,
-              edgesep: 0,
-              ranksep: 20,
-              rankdir: "LR",
-              marginx: 10,
-              marginy: 10
-            });
-            subgraphs[el.id] = sg;
-            loadJsonToDagre(sg, el);
-            r = new dagreD3.render();
-            d3tmpSvg.select('g').call(r, sg);
-            maxW = sg.graph().width;
-            maxH = sg.graph().height;
-            angular.element(mainTmpElement).empty();
-          }
-          createNode(g, data, el, isParent, maxW, maxH);
-          existingNodes.push(el.id);
-          if (el.inputs != null) {
-            ref = el.inputs;
-            for (l = 0, len1 = ref.length; l < len1; l++) {
-              pred = ref[l];
-              createEdge(g, data, el, existingNodes, pred, missingNodes);
-            }
-          }
-        }
-        return g;
-      };
-      searchForNode = function(data, nodeID) {
-        var el, i, j;
-        for (i in data.nodes) {
-          el = data.nodes[i];
-          if (el.id === nodeID) {
-            return el;
-          }
-          if (el.step_function != null) {
-            for (j in el.step_function) {
-              if (el.step_function[j].id === nodeID) {
-                return el.step_function[j];
-              }
-            }
-          }
-        }
-      };
-      drawGraph = function(data) {
-        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
-        g = new dagreD3.graphlib.Graph({
-          multigraph: true,
-          compound: true
-        }).setGraph({
-          nodesep: 70,
-          edgesep: 0,
-          ranksep: 50,
-          rankdir: "LR",
-          marginx: 40,
-          marginy: 40
-        });
-        loadJsonToDagre(g, data);
-        renderer = new dagreD3.render();
-        d3mainSvgG.call(renderer, g);
-        for (i in subgraphs) {
-          sg = subgraphs[i];
-          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
-        }
-        newScale = 0.5;
-        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
-        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
-        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
-        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
-        mainZoom.on("zoom", function() {
-          var ev;
-          ev = d3.event;
-          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
-        });
-        mainZoom(d3mainSvg);
-        return d3mainSvgG.selectAll('.node').on('click', function(d) {
-          return scope.setNode({
-            nodeid: d
-          });
-        });
-      };
-      scope.$watch(attrs.plan, function(newPlan) {
-        if (newPlan) {
-          return drawGraph(newPlan);
-        }
-      });
-    }
-  };
-}]);
-
-angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
-  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
-  currentJob = null;
-  currentPlan = null;
-  deferreds = {};
-  jobs = {
-    running: [],
-    finished: [],
-    cancelled: [],
-    failed: []
-  };
-  jobObservers = [];
-  notifyObservers = function() {
-    return angular.forEach(jobObservers, function(callback) {
-      return callback();
-    });
-  };
-  this.registerObserver = function(callback) {
-    return jobObservers.push(callback);
-  };
-  this.unRegisterObserver = function(callback) {
-    var index;
-    index = jobObservers.indexOf(callback);
-    return jobObservers.splice(index, 1);
-  };
-  this.stateList = function() {
-    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
-  };
-  this.translateLabelState = function(state) {
-    switch (state.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(list) {
-    return angular.forEach(list, function(item, jobKey) {
-      if (!(item['end-time'] > -1)) {
-        return item['end-time'] = item['start-time'] + item['duration'];
-      }
-    });
-  };
-  this.processVertices = function(data) {
-    angular.forEach(data.vertices, function(vertex, i) {
-      return vertex.type = 'regular';
-    });
-    return data.vertices.unshift({
-      name: 'Scheduled',
-      'start-time': data.timestamps['CREATED'],
-      'end-time': data.timestamps['CREATED'] + 1,
-      type: 'scheduled'
-    });
-  };
-  this.listJobs = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("/joboverview").success((function(_this) {
-      return function(data, status, headers, config) {
-        angular.forEach(data, function(list, listKey) {
-          switch (listKey) {
-            case 'running':
-              return jobs.running = _this.setEndTimes(list);
-            case 'finished':
-              return jobs.finished = _this.setEndTimes(list);
-            case 'cancelled':
-              return jobs.cancelled = _this.setEndTimes(list);
-            case 'failed':
-              return jobs.failed = _this.setEndTimes(list);
-          }
-        });
-        deferred.resolve(jobs);
-        return notifyObservers();
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getJobs = function(type) {
-    return jobs[type];
-  };
-  this.getAllJobs = function() {
-    return jobs;
-  };
-  this.loadJob = function(jobid) {
-    currentJob = null;
-    deferreds.job = $q.defer();
-    $http.get("/jobs/" + jobid).success((function(_this) {
-      return function(data, status, headers, config) {
-        _this.setEndTimes(data.vertices);
-        _this.processVertices(data);
-        return $http.get("/jobs/" + jobid + "/config").success(function(jobConfig) {
-          data = angular.extend(data, jobConfig);
-          currentJob = data;
-          return deferreds.job.resolve(currentJob);
-        });
-      };
-    })(this));
-    return deferreds.job.promise;
-  };
-  this.getNode = function(nodeid) {
-    var deferred, seekNode;
-    seekNode = function(nodeid, data) {
-      var j, len, node, sub;
-      for (j = 0, len = data.length; j < len; j++) {
-        node = data[j];
-        if (node.id === nodeid) {
-          return node;
-        }
-        if (node.step_function) {
-          sub = seekNode(nodeid, node.step_function);
-        }
-        if (sub) {
-          return sub;
-        }
-      }
-      return null;
-    };
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var foundNode;
-        foundNode = seekNode(nodeid, currentJob.plan.nodes);
-        foundNode.vertex = _this.seekVertex(nodeid);
-        return deferred.resolve(foundNode);
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.seekVertex = function(nodeid) {
-    var j, len, ref, vertex;
-    ref = currentJob.vertices;
-    for (j = 0, len = ref.length; j < len; j++) {
-      vertex = ref[j];
-      if (vertex.id === nodeid) {
-        return vertex;
-      }
-    }
-    return null;
-  };
-  this.getVertex = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var vertex;
-        vertex = _this.seekVertex(vertexid);
-        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
-          vertex.subtasks = data.subtasks;
-          return deferred.resolve(vertex);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getSubtasks = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
-          var subtasks;
-          subtasks = data.subtasks;
-          return deferred.resolve(subtasks);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getAccumulators = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
-          var accumulators;
-          accumulators = data['user-accumulators'];
-          return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
-            var subtaskAccumulators;
-            subtaskAccumulators = data.subtasks;
-            return deferred.resolve({
-              main: accumulators,
-              subtasks: subtaskAccumulators
-            });
-          });
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.loadExceptions = function() {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("/jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
-          currentJob.exceptions = exceptions;
-          return deferred.resolve(exceptions);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
-  var refresh;
-  $scope.jobObserver = function() {
-    $scope.runningJobs = JobsService.getJobs('running');
-    return $scope.finishedJobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  $scope.jobObserver();
-  OverviewService.loadOverview().then(function(data) {
-    return $scope.overview = data;
-  });
-  refresh = $interval(function() {
-    return OverviewService.loadOverview().then(function(data) {
-      return $scope.overview = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]);
-
-angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var overview;
-  overview = {};
-  this.loadOverview = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("/overview").success(function(data, status, headers, config) {
-      overview = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('TaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
-  var refresh;
-  TaskManagersService.loadManagers().then(function(data) {
-    return $scope.managers = data;
-  });
-  refresh = $interval(function() {
-    return TaskManagersService.loadManagers().then(function(data) {
-      return $scope.managers = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-  $scope.managerId = null;
-  return $scope.changeManager = function(managerId) {
-    if (managerId !== $scope.managerId) {
-      return $scope.managerId = managerId;
-    } else {
-      return $scope.managerId = null;
-    }
-  };
-}]);
-
-angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadManagers = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("/taskmanagers").success(function(data, status, headers, config) {
-      return deferred.resolve(data['taskmanagers']);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-<<<<<<< HEAD:flink-runtime-web/web-dashboard/web/js/index.js
-//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2Vycy90YXNrbWF
 uYWdlcnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2Vycy90YXNrbWFuYWdlcnMuY3RybC5qcyIsIm1vZHVsZXMvdGFza21hbmFnZXJzL3Rhc2ttYW5hZ2Vycy5zdmMuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlcnMvdGFza21hbmFnZXJzLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQzNCSCxPRDRCQSxZQUFZLGFBQWEsS0FBSyxTQUFDLFFBQUQ7SUFDNUIsUUFBUSxPQUFPLGFBQWE7SUFFNUIsWUFBWTtJQzVCWixPRDhCQSxVQUFVLFdBQUE7TUM3QlIsT0Q4QkEsWUFBWTtPQUNaLFlBQVk7O0lBS2pCLGlDQUFPLFNBQUMsdUJBQUQ7RUNoQ04sT0RpQ0Esc0JBQXNCO0lBSXZCLGdEQUFPLFNBQUMsZ0JBQWdCLG9CQUFqQjtFQUNOLGVBQWUsTUFBTSxZQUNuQjtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxnQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxrQkFDTDtJ
 QUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxjQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxtQkFDTDtJQUFBLEtBQUs7SUFDTCxVQUFVO0lBQ1YsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sNEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdDQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxnQkFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx1QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLDhCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxRQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxnQkFDT
 DtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxjQUNIO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTs7O0tBRXBCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7Ozs7RUNmbkIsT0RpQkEsbUJBQW1CLFVBQVU7O0FDZi9CO0FDaEpBLFFBQVEsT0FBTyxZQUlkLFVBQVUsMkJBQVcsU0FBQyxhQUFEO0VDckJwQixPRHNCQTtJQUFBLFlBQVk7SUFDWixTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLGlCQUFpQixZQUFZLG9CQUFvQixNQUFNOzs7O0lBSTVELFVBQVUsb0NBQW9CLFNBQUMsYUFBRDtFQ3JCN0IsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLHNDQUFzQyxZQUFZLG9CQUFvQixNQUFNOzs7O0lBSWpGLFVBQV
 UsaUJBQWlCLFdBQUE7RUNyQjFCLE9Ec0JBO0lBQUEsU0FBUztJQUNULE9BQ0U7TUFBQSxPQUFPOztJQUVULFVBQVU7OztBQ2xCWjtBQ3BCQSxRQUFRLE9BQU8sWUFFZCxPQUFPLG9EQUE0QixTQUFDLHFCQUFEO0VBQ2xDLElBQUE7RUFBQSxpQ0FBaUMsU0FBQyxPQUFPLFFBQVEsZ0JBQWhCO0lBQy9CLElBQWMsT0FBTyxVQUFTLGVBQWUsVUFBUyxNQUF0RDtNQUFBLE9BQU87O0lDaEJQLE9Ea0JBLE9BQU8sU0FBUyxPQUFPLFFBQVEsT0FBTyxnQkFBZ0I7TUFBRSxNQUFNOzs7RUFFaEUsK0JBQStCLFlBQVksb0JBQW9CO0VDZi9ELE9EaUJBO0lBRUQsT0FBTyxnQkFBZ0IsV0FBQTtFQ2pCdEIsT0RrQkEsU0FBQyxNQUFEO0lBRUUsSUFBRyxNQUFIO01DbEJFLE9Ea0JXLEtBQUssUUFBUSxTQUFTLEtBQUssUUFBUSxXQUFVO1dBQTFEO01DaEJFLE9EZ0JpRTs7O0dBRXRFLE9BQU8sU0FBUyxXQUFBO0VDZGYsT0RlQSxTQUFDLE9BQU8sV0FBUjtJQUNFLElBQUEsUUFBQTtJQUFBLElBQWUsTUFBTSxXQUFXLFdBQVcsQ0FBSSxTQUFTLFFBQXhEO01BQUEsT0FBTzs7SUFDUCxJQUFrQixPQUFPLGNBQWEsYUFBdEM7TUFBQSxZQUFZOztJQUNaLFFBQVEsQ0FBRSxTQUFTLE1BQU0sTUFBTSxNQUFNLE1BQU07SUFDM0MsU0FBUyxLQUFLLE1BQU0sS0FBSyxJQUFJLFNBQVMsS0FBSyxJQUFJO0lDVC9DLE9EVUEsQ0FBQyxRQUFRLEtBQUssSUFBSSxNQUFNLEtBQUssTUFBTSxVQUFVLFFBQVEsYUFBYSxNQUFNLE1BQU07OztBQ1BsRjtBQ2h
 CQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDhDQUFlLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ3RCLEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxXQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQ3BCUCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHNCQTs7QUNwQkY7QUNPQSxRQUFRLE9BQU8sWUFFZCxXQUFXLG9FQUE4QixTQUFDLFFBQVEseUJBQVQ7RUNuQnhDLE9Eb0JBLHdCQUF3QixhQUFhLEtBQUssU0FBQyxNQUFEO0lBQ3hDLElBQUksT0FBQSxjQUFBLE1BQUo7TUFDRSxPQUFPLGFBQWE7O0lDbEJ0QixPRG1CQSxPQUFPLFdBQVcsWUFBWTs7O0FDaEJsQztBQ1VBLFFBQVEsT0FBTyxZQUVkLFFBQVEsMERBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHNCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUNwQlQsT0RxQkEsU0FBUyxRQUFROztJQ25CbkIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7O0FDbkJGO0FDSUEsUUFBUSxPQUFPLFlBRWQsV0FBVyw2RUFBeUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUNuQyxPQUFPLGNBQWMsV0FBQTtJQ25CbkIsT0RvQkEsT0FBTyxPQUFPLFlBQVksUUFBUTs7RUFFcEMsWUFBWSxpQkFBaUIs
 T0FBTztFQUNwQyxPQUFPLElBQUksWUFBWSxXQUFBO0lDbkJyQixPRG9CQSxZQUFZLG1CQUFtQixPQUFPOztFQ2xCeEMsT0RvQkEsT0FBTztJQUlSLFdBQVcsK0VBQTJCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDckMsT0FBTyxjQUFjLFdBQUE7SUN0Qm5CLE9EdUJBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ3RCckIsT0R1QkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNyQnhDLE9EdUJBLE9BQU87SUFJUixXQUFXLHFIQUF1QixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQWEsWUFBWSxhQUFhLFdBQXJFO0VBQ2pDLElBQUE7RUFBQSxRQUFRLElBQUk7RUFFWixPQUFPLFFBQVEsYUFBYTtFQUM1QixPQUFPLE1BQU07RUFDYixPQUFPLE9BQU87RUFDZCxPQUFPLFdBQVc7RUFFbEIsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtJQUMzQyxPQUFPLE1BQU07SUFDYixPQUFPLE9BQU8sS0FBSztJQzFCbkIsT0QyQkEsT0FBTyxXQUFXLEtBQUs7O0VBRXpCLFlBQVksVUFBVSxXQUFBO0lDMUJwQixPRDJCQSxZQUFZLFFBQVEsYUFBYSxPQUFPLEtBQUssU0FBQyxNQUFEO01BQzNDLE9BQU8sTUFBTTtNQzFCYixPRDRCQSxPQUFPLFdBQVc7O0tBRXBCLFlBQVk7RUMzQmQsT0Q2QkEsT0FBTyxJQUFJLFlBQVksV0FBQTtJQUNyQixPQUFPLE1BQU07SUFDYixPQUFPLE9BQU87SUFDZCxPQUFPLFdBQVc7SUM1QmxCLE9EOEJBLFVBQVUsT
 0FBTzs7SUFLcEIsV0FBVyx5RUFBcUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUMvQixRQUFRLElBQUk7RUFFWixPQUFPLFNBQVM7RUFDaEIsT0FBTyxlQUFlO0VBQ3RCLE9BQU8sWUFBWSxZQUFZO0VBRS9CLE9BQU8sYUFBYSxTQUFDLFFBQUQ7SUFDbEIsSUFBRyxXQUFVLE9BQU8sUUFBcEI7TUFDRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxTQUFTO01BQ2hCLE9BQU8sV0FBVztNQUNsQixPQUFPLGVBQWU7TUNsQ3RCLE9Eb0NBLE9BQU8sV0FBVztXQU5wQjtNQVNFLE9BQU8sU0FBUztNQUNoQixPQUFPLGVBQWU7TUFDdEIsT0FBTyxTQUFTO01BQ2hCLE9BQU8sV0FBVztNQ3BDbEIsT0RxQ0EsT0FBTyxlQUFlOzs7RUFFMUIsT0FBTyxpQkFBaUIsV0FBQTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxlQUFlO0lBQ3RCLE9BQU8sU0FBUztJQUNoQixPQUFPLFdBQVc7SUNuQ2xCLE9Eb0NBLE9BQU8sZUFBZTs7RUNsQ3hCLE9Eb0NBLE9BQU8sYUFBYSxXQUFBO0lDbkNsQixPRG9DQSxPQUFPLGVBQWUsQ0FBQyxPQUFPOztJQUlqQyxXQUFXLHVEQUE2QixTQUFDLFFBQVEsYUFBVDtFQUN2QyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxLQUF2RDtJQUNFLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7TUN0QzFDLE9EdUNBLE9BQU8sV0FBVzs7O0VDcEN0QixPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0
 lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxZQUFZLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtRQ3JDMUMsT0RzQ0EsT0FBTyxXQUFXOzs7O0lBSXpCLFdBQVcsMkRBQWlDLFNBQUMsUUFBUSxhQUFUO0VBQzNDLFFBQVEsSUFBSTtFQUVaLElBQUcsT0FBTyxXQUFZLENBQUMsT0FBTyxVQUFVLENBQUMsT0FBTyxPQUFPLGVBQXZEO0lBQ0UsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01BQzlDLE9BQU8sZUFBZSxLQUFLO01DdEMzQixPRHVDQSxPQUFPLHNCQUFzQixLQUFLOzs7RUNwQ3RDLE9Ec0NBLE9BQU8sSUFBSSxVQUFVLFNBQUMsT0FBRDtJQUNuQixRQUFRLElBQUk7SUFDWixJQUFHLE9BQU8sUUFBVjtNQ3JDRSxPRHNDQSxZQUFZLGdCQUFnQixPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUFDOUMsT0FBTyxlQUFlLEtBQUs7UUNyQzNCLE9Ec0NBLE9BQU8sc0JBQXNCLEtBQUs7Ozs7SUFJekMsV0FBVyxtRkFBK0IsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUN6QyxRQUFRLElBQUk7RUFFWixZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO0lDdENoRCxPRHVDQSxPQUFPLFNBQVM7O0VDckNsQixPRHVDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lDdENaLE9EdUNBLFlBQVksVUFBVSxhQUFhLFVBQVUsS0FBSyxTQUFDLE1BQUQ7TUN0Q2hELE9EdUNBLE9BQU8sU0FBUzs7O0lBSXJCLFdBQVcsK0VBQTJCLFNBQUMsUUFBUSx
 RQUFRLGNBQWMsYUFBL0I7RUN2Q3JDLE9Ed0NBLFlBQVksaUJBQWlCLEtBQUssU0FBQyxNQUFEO0lDdkNoQyxPRHdDQSxPQUFPLGFBQWE7O0lBSXZCLFdBQVcscURBQTJCLFNBQUMsUUFBUSxhQUFUO0VBQ3JDLFFBQVEsSUFBSTtFQ3pDWixPRDJDQSxPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01DMUNoQixPRDRDQSxZQUFZLFFBQVEsUUFBUSxLQUFLLFNBQUMsTUFBRDtRQzNDL0IsT0Q0Q0EsT0FBTyxPQUFPOztXQUpsQjtNQU9FLE9BQU8sU0FBUztNQzNDaEIsT0Q0Q0EsT0FBTyxPQUFPOzs7O0FDeENwQjtBQ25IQSxRQUFRLE9BQU8sWUFJZCxVQUFVLHFCQUFVLFNBQUMsUUFBRDtFQ3JCbkIsT0RzQkE7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLE1BQU07O0lBRVIsTUFBTSxTQUFDLE9BQU8sTUFBTSxPQUFkO01BQ0osSUFBQSxhQUFBLFlBQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxLQUFLLFVBQVUsU0FBQyxTQUFTLEdBQVY7VUFDN0IsSUFBQTtVQUFBLFFBQVE7WUFDTjtjQUNFLE9BQU87Y0FDUCxPQUFPO2NBQ1AsYUFBYTtjQUNiLGVBQWUsUUFBUSxXQUFXO2NBQ2xDLGFBQWEsUUFBUSxXQUFXO2NB
 Q2hDLE1BQU07ZUFFUjtjQUNFLE9BQU87Y0FDUCxPQUFPO2NBQ1AsYUFBYTtjQUNiLGVBQWUsUUFBUSxXQUFXO2NBQ2xDLGFBQWEsUUFBUSxXQUFXO2NBQ2hDLE1BQU07OztVQUlWLElBQUcsUUFBUSxXQUFXLGNBQWMsR0FBcEM7WUFDRSxNQUFNLEtBQUs7Y0FDVCxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUN0QlIsT0R5QkYsU0FBUyxLQUFLO1lBQ1osT0FBTyxNQUFJLFFBQVEsVUFBUSxPQUFJLFFBQVE7WUFDdkMsT0FBTzs7O1FBR1gsUUFBUSxHQUFHLFdBQVcsUUFDckIsV0FBVztVQUNWLFFBQVEsR0FBRyxLQUFLLE9BQU87VUFFdkIsVUFBVTtXQUVYLE9BQU8sVUFDUCxZQUFZLFNBQUMsT0FBRDtVQzVCVCxPRDZCRjtXQUVELE9BQU87VUFBRSxNQUFNO1VBQUssT0FBTztVQUFHLEtBQUs7VUFBRyxRQUFRO1dBQzlDLFdBQVcsSUFDWDtRQzFCQyxPRDRCRixNQUFNLEdBQUcsT0FBTyxPQUNmLE1BQU0sVUFDTixLQUFLOztNQUVSLFlBQVksTUFBTTs7O0lBTXJCLFVBQVUsdUJBQVksU0FBQyxRQUFEO0VDaENyQixPRGlDQTtJQUFBLFVBQVU7SUFFVixPQUNFO01BQUEsVUFBVTtNQUNWLE9BQU87O0lBRVQsTUFBTSxTQUFDLE9BQU8sTUFBTSxPQUFkO01BQ0osSUFBQSxhQUFBLFlBQUEsT0FBQTtNQUFBLFFBQVEsS0FBSyxXQUFXO01BRXhCLGFBQWEsS0FBSztNQUNsQixRQUFRLFFBQVEsT0FBTyxLQUFLLFNBQVM7TUFFckMsa
 UJBQWlCLFNBQUMsT0FBRDtRQ2pDYixPRGtDRixNQUFNLFFBQVEsUUFBUTs7TUFFeEIsY0FBYyxTQUFDLE1BQUQ7UUFDWixJQUFBLE9BQUEsS0FBQTtRQUFBLEdBQUcsT0FBTyxPQUFPLFVBQVUsS0FBSztRQUVoQyxXQUFXO1FBRVgsUUFBUSxRQUFRLE1BQU0sU0FBQyxRQUFEO1VBQ3BCLElBQUcsT0FBTyxnQkFBZ0IsQ0FBQyxHQUEzQjtZQUNFLElBQUcsT0FBTyxTQUFRLGFBQWxCO2NDbENJLE9EbUNGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29CQUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTzs7OzttQkFSbkI7Y0NyQkksT0RnQ0YsU0FBUyxLQUNQO2dCQUFBLE9BQU87a0JBQ0w7b0JBQUEsT0FBTyxlQUFlLE9BQU87b0JBQzdCLE9BQU87b0JBQ1AsYUFBYTtvQkFDYixlQUFlLE9BQU87b0JBQ3RCLGFBQWEsT0FBTztvQkFDcEIsTUFBTSxPQUFPO29CQUNiLE1BQU0sT0FBTzs7Ozs7OztRQUd2QixRQUFRLEdBQUcsV0FBVyxRQUFRLE1BQU0sU0FBQyxHQUFHLEdBQUcsT0FBUDtVQUNsQyxJQUFHLEVBQUUsTUFBTDtZQzFCSSxPRDJCRixPQUFPLEdBQUcsOEJBQThCO2NBQUUsT0FBTyxNQUFNO2NBQU8sVUFBVSxFQUFFOzs7V0FHN0UsV0FBVztVQUNWLFFBQVEsR0FBRyxLQUFLLE9BQU87VUFHdkIsVUFBVTtXQUVYLE9BQU8sUUFDUCxPQUFPO1VBQUUsTUFBTTtVQUFHLE9BQU87VUFBRyxLQUFLO1VBQU
 csUUFBUTtXQUM1QyxXQUFXLElBQ1gsaUJBQ0E7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixNQUFNLE9BQU8sTUFBTSxVQUFVLFNBQUMsTUFBRDtRQUMzQixJQUFxQixNQUFyQjtVQzdCSSxPRDZCSixZQUFZOzs7OztJQU1qQixVQUFVLHdCQUFXLFNBQUMsVUFBRDtFQzdCcEIsT0Q4QkE7SUFBQSxVQUFVO0lBUVYsT0FDRTtNQUFBLE1BQU07TUFDTixTQUFTOztJQUVYLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsWUFBQSxZQUFBLGlCQUFBLGlCQUFBLFlBQUEsV0FBQSxZQUFBLFVBQUEsV0FBQSw2QkFBQSxHQUFBLGFBQUEsd0JBQUEsT0FBQSxpQkFBQSxPQUFBLGdCQUFBLGdCQUFBLFVBQUEsZUFBQSxlQUFBO01BQUEsSUFBSTtNQUNKLFdBQVcsR0FBRyxTQUFTO01BQ3ZCLFlBQVk7TUFDWixRQUFRLE1BQU07TUFFZCxpQkFBaUIsS0FBSyxXQUFXO01BQ2pDLFFBQVEsS0FBSyxXQUFXLFdBQVc7TUFDbkMsaUJBQWlCLEtBQUssV0FBVztNQUVqQyxZQUFZLEdBQUcsT0FBTztNQUN0QixhQUFhLEdBQUcsT0FBTztNQUN2QixXQUFXLEdBQUcsT0FBTztNQUtyQixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLEtBQUssV0FBVyxJQUFJLE1BQU07TUFFMUMsTUFBTSxTQUFTLFdBQUE7UUFDYixJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsWUFBWSxTQUFTO1VBQ3JCLEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM
 7VUFDeEQsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFNBQVMsVUFBVSxDQUFFLElBQUk7VUMxQ3ZCLE9ENkNGLFdBQVcsS0FBSyxhQUFhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUVoRyxNQUFNLFVBQVUsV0FBQTtRQUNkLElBQUEsV0FBQSxJQUFBO1FBQUEsSUFBRyxTQUFTLFVBQVUsTUFBdEI7VUFHRSxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxVQUFVLENBQUUsSUFBSTtVQzVDdkIsT0QrQ0YsV0FBVyxLQUFLLGFBQWEsZUFBZSxLQUFLLE1BQU0sS0FBSyxhQUFhLFNBQVMsVUFBVTs7O01BR2hHLGtCQUFrQixTQUFDLElBQUQ7UUFDaEIsSUFBQTtRQUFBLGFBQWE7UUFDYixJQUFHLENBQUEsR0FBQSxpQkFBQSxVQUFxQixHQUFBLGtCQUFBLE9BQXhCO1VBQ0UsY0FBYztVQUNkLElBQW1DLEdBQUEsaUJBQUEsTUFBbkM7WUFBQSxjQUFjLEdBQUc7O1VBQ2pCLElBQWdELEdBQUcsY0FBYSxXQUFoRTtZQUFBLGNBQWMsT0FBTyxHQUFHLFlBQVk7O1VBQ3BDLElBQWtELEdBQUcsbUJBQWtCLFdBQXZFO1lBQUEsY0FBYyxVQUFVLEdBQUc7O1VBQzNCLGNBQWM7O1FDdENkLE9EdUNGOztNQUlGLHlCQUF5QixTQUFDLE1BQUQ7UUN4Q3JCLE9E
 eUNELFNBQVEscUJBQXFCLFNBQVEseUJBQXlCLFNBQVEsYUFBYSxTQUFRLGlCQUFpQixTQUFRLGlCQUFpQixTQUFROztNQUVoSixjQUFjLFNBQUMsSUFBSSxNQUFMO1FBQ1osSUFBRyxTQUFRLFVBQVg7VUN4Q0ksT0R5Q0Y7ZUFFRyxJQUFHLHVCQUF1QixPQUExQjtVQ3pDRCxPRDBDRjtlQURHO1VDdkNELE9EMkNBOzs7TUFHTixrQkFBa0IsU0FBQyxJQUFJLE1BQU0sTUFBTSxNQUFqQjtRQUVoQixJQUFBLFlBQUE7UUFBQSxhQUFhLHVCQUF1QixRQUFRLGFBQWEsR0FBRyxLQUFLLHlCQUF5QixZQUFZLElBQUksUUFBUTtRQUdsSCxJQUFHLFNBQVEsVUFBWDtVQUNFLGNBQWMscUNBQXFDLEdBQUcsV0FBVztlQURuRTtVQUdFLGNBQWMsMkJBQTJCLEdBQUcsV0FBVzs7UUFDekQsSUFBRyxHQUFHLGdCQUFlLElBQXJCO1VBQ0UsY0FBYztlQURoQjtVQUdFLFdBQVcsR0FBRztVQUdkLFdBQVcsY0FBYztVQUN6QixjQUFjLDJCQUEyQixXQUFXOztRQUd0RCxJQUFHLEdBQUEsaUJBQUEsTUFBSDtVQUNFLGNBQWMsNEJBQTRCLEdBQUcsSUFBSSxNQUFNO2VBRHpEO1VBS0UsSUFBK0MsdUJBQXVCLE9BQXRFO1lBQUEsY0FBYyxTQUFTLE9BQU87O1VBQzlCLElBQXFFLEdBQUcsZ0JBQWUsSUFBdkY7WUFBQSxjQUFjLHNCQUFzQixHQUFHLGNBQWM7O1VBQ3JELElBQXdGLEdBQUcsYUFBWSxXQUF2RztZQUFBLGNBQWMsb0JBQW9CLGNBQWMsR0FBRyxxQkFBcUI7OztRQUcxRSxjQUFjO1FDM0NaLE9ENENGOztNQUdGLDhCQUE4QixTQUFDLElBQ
 UksTUFBTSxNQUFYO1FBQzVCLElBQUEsWUFBQTtRQUFBLFFBQVEsU0FBUztRQUVqQixhQUFhLGlCQUFpQixRQUFRLGFBQWEsT0FBTyxhQUFhLE9BQU87UUM1QzVFLE9ENkNGOztNQUdGLGdCQUFnQixTQUFDLEdBQUQ7UUFFZCxJQUFBO1FBQUEsSUFBRyxFQUFFLE9BQU8sT0FBTSxLQUFsQjtVQUNFLElBQUksRUFBRSxRQUFRLEtBQUs7VUFDbkIsSUFBSSxFQUFFLFFBQVEsS0FBSzs7UUFDckIsTUFBTTtRQUNOLE9BQU0sRUFBRSxTQUFTLElBQWpCO1VBQ0UsTUFBTSxNQUFNLEVBQUUsVUFBVSxHQUFHLE1BQU07VUFDakMsSUFBSSxFQUFFLFVBQVUsSUFBSSxFQUFFOztRQUN4QixNQUFNLE1BQU07UUMzQ1YsT0Q0Q0Y7O01BRUYsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLFVBQWtCLE1BQU0sTUFBdEM7UUMzQ1QsSUFBSSxZQUFZLE1BQU07VUQyQ0MsV0FBVzs7UUFFcEMsSUFBRyxHQUFHLE9BQU0sS0FBSyxrQkFBakI7VUN6Q0ksT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksbUJBQW1CLE1BQU07WUFDcEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLHVCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSx1QkFBdUIsTUFBTTtZQUN4RCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssU0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQU
 csSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksV0FBVyxNQUFNO1lBQzVDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGVBQWUsTUFBTTtZQUNoRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssZ0JBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGlCQUFpQixNQUFNO1lBQ2xELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFKdEI7VUNuQ0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksSUFBSSxNQUFNO1lBQ3JDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7OztNQUU3QixhQUFhLFNBQUMsR0FBRyxNQUFNLElBQUksZUFBZSxNQUFNLGNBQW5DO1FBQ1gsSUFBQTtRQUFBLElBQU8sY0FBYyxRQUFRLEtBQUssUUFBTyxDQUFDLEdBQTFDO1VDdENJLE9EdUNGLEVBQUUsUUFBUSxLQUFLLElBQUksR0FBRyxJQUNwQjtZQUFBLE9BQU8sZ0JBQWdCO1lBQ3ZCLFdBQVc7WUFDWCxXQUFXOztlQUpmO1V
 BT0UsY0FBYyxjQUFjLE1BQU0sS0FBSztVQUV2QyxJQUFBLEVBQU8sQ0FBQyxlQUFlLGFBQWEsUUFBUSxZQUFZLE1BQU0sQ0FBQyxJQUEvRDtZQUNFLGFBQWEsS0FBSyxZQUFZO1lBQzlCLEVBQUUsUUFBUSxZQUFZLElBQ3BCO2NBQUEsT0FBTyxnQkFBZ0IsYUFBYTtjQUNwQyxXQUFXO2NBQ1gsU0FBTyxZQUFZLGFBQWE7O1lDdENoQyxPRHdDRixFQUFFLFFBQVEsWUFBWSxJQUFJLEdBQUcsSUFDM0I7Y0FBQSxPQUFPLGdCQUFnQjtjQUN2QixXQUFXOzs7OztNQUVuQixrQkFBa0IsU0FBQyxHQUFHLE1BQUo7UUFDaEIsSUFBQSxJQUFBLGVBQUEsVUFBQSxHQUFBLEdBQUEsS0FBQSxNQUFBLE1BQUEsTUFBQSxjQUFBLE1BQUEsR0FBQSxLQUFBLElBQUE7UUFBQSxnQkFBZ0I7UUFDaEIsZUFBZTtRQUVmLElBQUcsS0FBQSxTQUFBLE1BQUg7VUFFRSxZQUFZLEtBQUs7ZUFGbkI7VUFNRSxZQUFZLEtBQUs7VUFDakIsV0FBVzs7UUFFYixLQUFBLElBQUEsR0FBQSxNQUFBLFVBQUEsUUFBQSxJQUFBLEtBQUEsS0FBQTtVQ3ZDSSxLQUFLLFVBQVU7VUR3Q2pCLE9BQU87VUFDUCxPQUFPO1VBRVAsSUFBRyxHQUFHLGVBQU47WUFDRSxLQUFTLElBQUEsUUFBUSxTQUFTLE1BQU07Y0FBRSxZQUFZO2NBQU0sVUFBVTtlQUFRLFNBQVM7Y0FDN0UsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTOztZQUdYLFVBQVUsR0FBRyxNQUFNO1lBRW5CLGdCQUFnQixJQUFJO1lBRXBCLElBQVEsSUFBQSxR
 QUFRO1lBQ2hCLFNBQVMsT0FBTyxLQUFLLEtBQUssR0FBRztZQUM3QixPQUFPLEdBQUcsUUFBUTtZQUNsQixPQUFPLEdBQUcsUUFBUTtZQUVsQixRQUFRLFFBQVEsZ0JBQWdCOztVQUVsQyxXQUFXLEdBQUcsTUFBTSxJQUFJLFVBQVUsTUFBTTtVQUV4QyxjQUFjLEtBQUssR0FBRztVQUd0QixJQUFHLEdBQUEsVUFBQSxNQUFIO1lBQ0UsTUFBQSxHQUFBO1lBQUEsS0FBQSxJQUFBLEdBQUEsT0FBQSxJQUFBLFFBQUEsSUFBQSxNQUFBLEtBQUE7Y0MxQ0ksT0FBTyxJQUFJO2NEMkNiLFdBQVcsR0FBRyxNQUFNLElBQUksZUFBZSxNQUFNOzs7O1FDdENqRCxPRHdDRjs7TUFHRixnQkFBZ0IsU0FBQyxNQUFNLFFBQVA7UUFDZCxJQUFBLElBQUEsR0FBQTtRQUFBLEtBQUEsS0FBQSxLQUFBLE9BQUE7VUFDRSxLQUFLLEtBQUssTUFBTTtVQUNoQixJQUFjLEdBQUcsT0FBTSxRQUF2QjtZQUFBLE9BQU87O1VBR1AsSUFBRyxHQUFBLGlCQUFBLE1BQUg7WUFDRSxLQUFBLEtBQUEsR0FBQSxlQUFBO2NBQ0UsSUFBK0IsR0FBRyxjQUFjLEdBQUcsT0FBTSxRQUF6RDtnQkFBQSxPQUFPLEdBQUcsY0FBYzs7Ozs7O01BRWhDLFlBQVksU0FBQyxNQUFEO1FBQ1YsSUFBQSxHQUFBLFVBQUEsVUFBQSxJQUFBLGVBQUE7UUFBQSxJQUFRLElBQUEsUUFBUSxTQUFTLE1BQU07VUFBRSxZQUFZO1VBQU0sVUFBVTtXQUFRLFNBQVM7VUFDNUUsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTOztRQUdYLGdCQ
 UFnQixHQUFHO1FBRW5CLFdBQWUsSUFBQSxRQUFRO1FBQ3ZCLFdBQVcsS0FBSyxVQUFVO1FBRTFCLEtBQUEsS0FBQSxXQUFBO1VDakNJLEtBQUssVUFBVTtVRGtDakIsVUFBVSxPQUFPLGFBQWEsSUFBSSxNQUFNLEtBQUssVUFBVTs7UUFFekQsV0FBVztRQUVYLGdCQUFnQixLQUFLLE1BQU0sQ0FBQyxRQUFRLFFBQVEsZ0JBQWdCLFVBQVUsRUFBRSxRQUFRLFFBQVEsWUFBWTtRQUNwRyxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixXQUFXLEVBQUUsUUFBUSxTQUFTLFlBQVk7UUFFdEcsU0FBUyxNQUFNLFVBQVUsVUFBVSxDQUFDLGVBQWU7UUFFbkQsV0FBVyxLQUFLLGFBQWEsZUFBZSxnQkFBZ0IsT0FBTyxnQkFBZ0IsYUFBYSxTQUFTLFVBQVU7UUFFbkgsU0FBUyxHQUFHLFFBQVEsV0FBQTtVQUNsQixJQUFBO1VBQUEsS0FBSyxHQUFHO1VDbkNOLE9Eb0NGLFdBQVcsS0FBSyxhQUFhLGVBQWUsR0FBRyxZQUFZLGFBQWEsR0FBRyxRQUFROztRQUVyRixTQUFTO1FDbkNQLE9EcUNGLFdBQVcsVUFBVSxTQUFTLEdBQUcsU0FBUyxTQUFDLEdBQUQ7VUNwQ3RDLE9EcUNGLE1BQU0sUUFBUTtZQUFFLFFBQVE7Ozs7TUFFNUIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLFNBQUQ7UUFDdkIsSUFBc0IsU0FBdEI7VUNqQ0ksT0RpQ0osVUFBVTs7Ozs7O0FDM0JoQjtBQzFhQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDhFQUFlLFNBQUMsT0FBTyxhQUFhLE1BQU0sVUFBVSxJQUFJLFVBQXpDO0VBQ3RCLElBQUEsWUFBQS
 xhQUFBLFdBQUEsY0FBQSxNQUFBO0VBQUEsYUFBYTtFQUNiLGNBQWM7RUFFZCxZQUFZO0VBQ1osT0FBTztJQUNMLFNBQVM7SUFDVCxVQUFVO0lBQ1YsV0FBVztJQUNYLFFBQVE7O0VBR1YsZUFBZTtFQUVmLGtCQUFrQixXQUFBO0lDckJoQixPRHNCQSxRQUFRLFFBQVEsY0FBYyxTQUFDLFVBQUQ7TUNyQjVCLE9Ec0JBOzs7RUFFSixLQUFDLG1CQUFtQixTQUFDLFVBQUQ7SUNwQmxCLE9EcUJBLGFBQWEsS0FBSzs7RUFFcEIsS0FBQyxxQkFBcUIsU0FBQyxVQUFEO0lBQ3BCLElBQUE7SUFBQSxRQUFRLGFBQWEsUUFBUTtJQ25CN0IsT0RvQkEsYUFBYSxPQUFPLE9BQU87O0VBRTdCLEtBQUMsWUFBWSxXQUFBO0lDbkJYLE9Eb0JBLENBRUUsYUFDQSxhQUNBLFdBQ0EsWUFDQSxVQUNBLGFBQ0E7O0VBR0osS0FBQyxzQkFBc0IsU0FBQyxPQUFEO0lBQ3JCLFFBQU8sTUFBTTtNQUFiLEtBQ087UUM1QkgsT0Q0Qm1CO01BRHZCLEtBRU87UUMzQkgsT0QyQmlCO01BRnJCLEtBR087UUMxQkgsT0QwQm9CO01BSHhCLEtBSU87UUN6QkgsT0R5Qm9CO01BSnhCLEtBS087UUN4QkgsT0R3QmtCO01BTHRCLEtBTU87UUN2QkgsT0R1Qm9CO01BTnhCLEtBT087UUN0QkgsT0RzQmtCO01BUHRCLEtBUU87UUNyQkgsT0RxQmdCO01BUnBCO1FDWEksT0RvQkc7OztFQUVULEtBQUMsY0FBYyxTQUFDLE1BQUQ7SUNsQmIsT0RtQkEsUUFBUSxRQUFRLE1BQU0sU0FBQyxNQUFNLFFBQVA7TUFDcEIsSUFBQSxFQUFPLEtBQUssY0FBYyxDQUFDLElBQTNCO1F
 DbEJFLE9EbUJBLEtBQUssY0FBYyxLQUFLLGdCQUFnQixLQUFLOzs7O0VBRW5ELEtBQUMsa0JBQWtCLFNBQUMsTUFBRDtJQUNqQixRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsUUFBUSxHQUFUO01DaEI3QixPRGlCQSxPQUFPLE9BQU87O0lDZmhCLE9EaUJBLEtBQUssU0FBUyxRQUFRO01BQ3BCLE1BQU07TUFDTixjQUFjLEtBQUssV0FBVztNQUM5QixZQUFZLEtBQUssV0FBVyxhQUFhO01BQ3pDLE1BQU07OztFQUdWLEtBQUMsV0FBVyxXQUFBO0lBQ1YsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxnQkFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lB
 RW5CLE1BQU0sSUFBSSxXQUFXLE9BQ3BCLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFdBQVcsUUFBUSxXQUM1QixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQ
 yxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxXQUFXLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQy9ELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RFRixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsVUFDcEQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSS
 xRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDL0QsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDL0QsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ESnlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxXQUFXLFdBQVcsTUFBTSxlQUNyQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVk
 sV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLGFBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLHdGQUEwQixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDcEMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VBRWQsT0FBTyxJQUFJLFlBQVks
 V0FBQTtJQ2xCckIsT0RtQkEsVUFBVSxPQUFPOztFQUVuQixPQUFPLFlBQVk7RUNsQm5CLE9Eb0JBLE9BQU8sZ0JBQWdCLFNBQUMsV0FBRDtJQUNyQixJQUFHLGNBQWEsT0FBTyxXQUF2QjtNQ25CRSxPRG9CQSxPQUFPLFlBQVk7V0FEckI7TUNqQkUsT0RvQkEsT0FBTyxZQUFZOzs7O0FDaEJ6QjtBQ0pBLFFBQVEsT0FBTyxZQUVkLFFBQVEsc0RBQXVCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzlCLEtBQUMsZUFBZSxXQUFBO0lBQ2QsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxpQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUNwQlAsT0RxQkEsU0FBUyxRQUFRLEtBQUs7O0lDbkJ4QixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkYiLCJmaWxlIjoiaW5kZXguanMiLCJzb3VyY2VzQ29udGVudCI6WyIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgb
 m90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKCRyb290U2NvcGUpIC0+XG4gICRyb290U2NvcGUuc2lkZWJhclZpc2libGUgPSBmYWxzZVxuICAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gLT5cbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcbiAgICAkcm
 9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93J1xuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi52YWx1ZSAnZmxpbmtDb25maWcnLCB7XG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxufVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKEpvYnNTZXJ2aWNlLCBNYWluU2VydmljZSwgZmxpbmtDb25maWcsICRpbnRlcnZhbCkgLT5cbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cbiAgICBhbmd1bGFyLmV4dGVuZCBmbGlua0NvbmZpZywgY29uZmlnXG5cbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXG5cbiAgICAkaW50ZXJ2YWwgLT5cbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcbiAgICAsIGZsaW5rQ29uZmlnW1wicmVmcmVzaC1pbnRlcnZhbFwiXVxuXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxuICAkdWlWaWV3U2Nyb2xsUHJvdmlkZXIudXNlQW5jaG9yU2Nyb2xsKClcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29uZmlnICgkc3RhdGVQcm92aWRlciwgJHVybFJvdXRlclByb3ZpZGVyKSAtPlxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXG4gICAgdXJsOiBcIi9vdmVydmlld1wiXG4gICAgdmlld3M
 6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9vdmVydmlldy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ092ZXJ2aWV3Q29udHJvbGxlcidcblxuICAuc3RhdGUgXCJydW5uaW5nLWpvYnNcIixcbiAgICB1cmw6IFwiL3J1bm5pbmctam9ic1wiXG4gICAgdmlld3M6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL3J1bm5pbmctam9icy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcbiAgXG4gIC5zdGF0ZSBcImNvbXBsZXRlZC1qb2JzXCIsXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXG4gICAgdmlld3M6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnQ29tcGxldGVkSm9ic0NvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxuICAgIHVybDogXCIvam9icy97am9iaWR9XCJcbiAgICBhYnN0cmFjdDogdHJ1ZVxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IucGxhblwiLFxuICAgIHVybDogXCJcIlxuICAgIGFic3RyYWN0OiB0cnVl
 XG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkNvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLm92ZXJ2aWV3XCIsXG4gICAgdXJsOiBcIlwiXG4gICAgdmlld3M6XG4gICAgICAnbm9kZS1kZXRhaWxzJzpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3Qub3ZlcnZpZXcuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4uYWNjdW11bGF0b3JzXCIsXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxuICAgIHZpZXdzOlxuICAgICAgJ25vZGUtZGV0YWlscyc6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJyBcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXG4gICAgdXJsOiBcIi90aW1lbGluZVwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUta
 m9iLnRpbWVsaW5lLnZlcnRleFwiLFxuICAgIHVybDogXCIve3ZlcnRleElkfVwiXG4gICAgdmlld3M6XG4gICAgICB2ZXJ0ZXg6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnN0YXRpc3RpY3NcIixcbiAgICB1cmw6IFwiL3N0YXRpc3RpY3NcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLmV4Y2VwdGlvbnNcIixcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuZXhjZXB0aW9ucy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IucHJvcGVydGllc1wiLFxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUHJvcGVydGllc0NvbnRyb2xsZXInXG5cbi
 AgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcbiAgICB1cmw6IFwiL2NvbmZpZ1wiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5jb25maWcuaHRtbFwiXG5cbiAgLnN0YXRlIFwidGFza21hbmFnZXJzXCIsXG4gICAgdXJsOiBcIi90YXNrbWFuYWdlcnNcIlxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvdGFza21hbmFnZXJzL2luZGV4Lmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnVGFza01hbmFnZXJzQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyXCIsXG4gICAgICB1cmw6IFwiL2pvYm1hbmFnZXJcIlxuICAgICAgdmlld3M6XG4gICAgICAgIG1haW46XG4gICAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9pbmRleC5odG1sXCJcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyLmNvbmZpZ1wiLFxuICAgIHVybDogXCIvY29uZmlnXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvY29uZmlnLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iTWFuYWdlckNvbmZpZ0NvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwiam9ibWFuYWdlci5zdGRvdXRcIixcbiAgICB1cmw6IFwiL3N0ZG91dFwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlx
 uICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL3N0ZG91dC5odG1sXCJcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyLmxvZ2ZpbGVcIixcbiAgICB1cmw6IFwiL2xvZ2ZpbGVcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9sb2dmaWxlLmh0bWxcIlxuXG4gICR1cmxSb3V0ZXJQcm92aWRlci5vdGhlcndpc2UgXCIvb3ZlcnZpZXdcIlxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKS5ydW4oZnVuY3Rpb24oJHJvb3RTY29wZSkge1xuICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gZmFsc2U7XG4gIHJldHVybiAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gZnVuY3Rpb24oKSB7XG4gICAgJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZSA9ICEkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlO1xuICAgIHJldHVybiAkcm9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93JztcbiAgfTtcbn0pLnZhbHVlKCdmbGlua0NvbmZpZycsIHtcbiAgXCJyZWZyZXNoLWludGVydmFsXCI6IDEwMDAwXG59KS5ydW4oZnVuY3Rpb24oSm9ic1NlcnZpY2UsIE1haW5TZXJ2aWNlLCBmbGlua0NvbmZpZywgJGludGVydmFsKSB7XG4gIHJldHVybiBNYWluU2VydmljZS5sb2FkQ29uZmlnKCkudGhlbihmdW5jdGlvbihjb25maWcpIHtcbiAgICBhbmd1bGFyLmV4
 dGVuZChmbGlua0NvbmZpZywgY29uZmlnKTtcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpO1xuICAgIHJldHVybiAkaW50ZXJ2YWwoZnVuY3Rpb24oKSB7XG4gICAgICByZXR1cm4gSm9ic1NlcnZpY2UubGlzdEpvYnMoKTtcbiAgICB9LCBmbGlua0NvbmZpZ1tcInJlZnJlc2gtaW50ZXJ2YWxcIl0pO1xuICB9KTtcbn0pLmNvbmZpZyhmdW5jdGlvbigkdWlWaWV3U2Nyb2xsUHJvdmlkZXIpIHtcbiAgcmV0dXJuICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKTtcbn0pLmNvbmZpZyhmdW5jdGlvbigkc3RhdGVQcm92aWRlciwgJHVybFJvdXRlclByb3ZpZGVyKSB7XG4gICRzdGF0ZVByb3ZpZGVyLnN0YXRlKFwib3ZlcnZpZXdcIiwge1xuICAgIHVybDogXCIvb3ZlcnZpZXdcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9vdmVydmlldy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInJ1bm5pbmctam9ic1wiLCB7XG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL3J1bm5pbmctam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdSdW5uaW5nSm9ic0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuI
 CB9KS5zdGF0ZShcImNvbXBsZXRlZC1qb2JzXCIsIHtcbiAgICB1cmw6IFwiL2NvbXBsZXRlZC1qb2JzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIG1haW46IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9jb21wbGV0ZWQtam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYlwiLCB7XG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIixcbiAgICBhYnN0cmFjdDogdHJ1ZSxcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW5cIiwge1xuICAgIHVybDogXCJcIixcbiAgICBhYnN0cmFjdDogdHJ1ZSxcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIiwge1xuICAgIHVybDogXCJcIixcbiAgICB2aWV3czoge1xuICAgICAgJ25vZGUtZGV0YWlscyc6IH
 tcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3Qub3ZlcnZpZXcuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbk92ZXJ2aWV3Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLCB7XG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIixcbiAgICB2aWV3czoge1xuICAgICAgJ25vZGUtZGV0YWlscyc6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3QuYWNjdW11bGF0b3JzLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsIHtcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IudGltZWxpbmUudmVydGV4XCIsIHtcbiAgICB1cmw6IFwiL3t2ZXJ0ZXhJZH1cIixcbiAgICB2aWV3czoge1xuICAgICAgdmVydGV4OiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCIsXG4gICA
 gICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLCB7XG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5leGNlcHRpb25zXCIsIHtcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5leGNlcHRpb25zLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIiwge1xuICAgIHVybDogXCIvcHJvcGVydGllc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnByb3BlcnRpZXMuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUHJvcGVydGllc0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IuY29uZmlnXCIsIHtcbiAgICB1cmw6IFwiL2NvbmZpZ1wiLFxu
 ICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwidGFza21hbmFnZXJzXCIsIHtcbiAgICB1cmw6IFwiL3Rhc2ttYW5hZ2Vyc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2Vycy9pbmRleC5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJqb2JtYW5hZ2VyXCIsIHtcbiAgICB1cmw6IFwiL2pvYm1hbmFnZXJcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL2luZGV4Lmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJqb2JtYW5hZ2VyLmNvbmZpZ1wiLCB7XG4gICAgdXJsOiBcIi9jb25maWdcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL2NvbmZpZy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyQ29uZmlnQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiam9ibWFuYWdlci5zdGRvdXRcIiwge1xuICAgIHVybDogXCIvc3Rkb3V0X
 CIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9zdGRvdXQuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcImpvYm1hbmFnZXIubG9nZmlsZVwiLCB7XG4gICAgdXJsOiBcIi9sb2dmaWxlXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9sb2dmaWxlLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSk7XG4gIHJldHVybiAkdXJsUm91dGVyUHJvdmlkZXIub3RoZXJ3aXNlKFwiL292ZXJ2aWV3XCIpO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YW
 luIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ2JzTGFiZWwnLCAoSm9ic1NlcnZpY2UpIC0+XG4gIHRyYW5zY2x1ZGU6IHRydWVcbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6IFwiQFwiXG5cbiAgdGVtcGxhdGU6IFwiPHNwYW4gdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknPjxuZy10cmFuc2NsdWRlPjwvbmctdHJhbnNjbHVkZT48L3NwYW4+XCJcbiAgXG4gIGxpbms
 6IChzY29wZSwgZWxlbWVudCwgYXR0cnMpIC0+XG4gICAgc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IC0+XG4gICAgICAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmRpcmVjdGl2ZSAnaW5kaWNhdG9yUHJpbWFyeScsIChKb2JzU2VydmljZSkgLT5cbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6ICdAJ1xuXG4gIHRlbXBsYXRlOiBcIjxpIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJyAvPlwiXG4gIFxuICBsaW5rOiAoc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSAtPlxuICAgIHNjb3BlLmdldExhYmVsQ2xhc3MgPSAtPlxuICAgICAgJ2ZhIGZhLWNpcmNsZSBpbmRpY2F0b3IgaW5kaWNhdG9yLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ3RhYmxlUHJvcGVydHknLCAtPlxuICByZXBsYWNlOiB0cnVlXG4gIHNjb3BlOlxuICAgIHZhbHVlOiAnPSdcblxuICB0ZW1wbGF0ZTogXCI8dGQgdGl0bGU9XFxcInt7dmFsdWUgfHwgJ05vbmUnfX1cXFwiPnt7dmFsdWUgfHwgJ05vbmUnfX08L3RkPlwiXG4i
 LCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5kaXJlY3RpdmUoJ2JzTGFiZWwnLCBmdW5jdGlvbihKb2JzU2VydmljZSkge1xuICByZXR1cm4ge1xuICAgIHRyYW5zY2x1ZGU6IHRydWUsXG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgZ2V0TGFiZWxDbGFzczogXCImXCIsXG4gICAgICBzdGF0dXM6IFwiQFwiXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8c3BhbiB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKSc+PG5nLXRyYW5zY2x1ZGU+PC9uZy10cmFuc2NsdWRlPjwvc3Bhbj5cIixcbiAgICBsaW5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCdpbmRpY2F0b3JQcmltYXJ5JywgZnVuY3Rpb24oSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIHtcbiAgICByZXBsYWNlOiB0cnVlLFxuICAgIHNjb3BlOiB7XG4gICAgICBnZXRMYWJlbENsYXNzOiBcIiZcIixcbiAgICAgIHN0YXR1czogJ0AnXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8aSB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKScgLz5cIixcbiAgICBsa
 W5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnZmEgZmEtY2lyY2xlIGluZGljYXRvciBpbmRpY2F0b3ItJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCd0YWJsZVByb3BlcnR5JywgZnVuY3Rpb24oKSB7XG4gIHJldHVybiB7XG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgdmFsdWU6ICc9J1xuICAgIH0sXG4gICAgdGVtcGxhdGU6IFwiPHRkIHRpdGxlPVxcXCJ7e3ZhbHVlIHx8ICdOb25lJ319XFxcIj57e3ZhbHVlIHx8ICdOb25lJ319PC90ZD5cIlxuICB9O1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYX
 kgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmZpbHRlciBcImFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZFwiLCAoYW5ndWxhck1vbWVudENvbmZpZykgLT5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gKHZhbHVlLCBmb3JtYXQsIGR1cmF0aW9uRm9ybWF0KSAtPlxuICAgIHJldHVybiBcIlwiICBpZiB0eXBlb2YgdmFsdWUgaXMgXCJ1bmRlZmluZWRcIiBvciB2YWx1ZSBpcyBudWxsXG5cbiAgICBtb21lbnQuZHVyYXRpb24odmFsdWU
 sIGZvcm1hdCkuZm9ybWF0KGR1cmF0aW9uRm9ybWF0LCB7IHRyaW06IGZhbHNlIH0pXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyXG5cbi5maWx0ZXIgXCJodW1hbml6ZVRleHRcIiwgLT5cbiAgKHRleHQpIC0+XG4gICAgIyBUT0RPOiBleHRlbmQuLi4gYSBsb3RcbiAgICBpZiB0ZXh0IHRoZW4gdGV4dC5yZXBsYWNlKC8mZ3Q7L2csIFwiPlwiKS5yZXBsYWNlKC88YnJcXC8+L2csXCJcIikgZWxzZSAnJ1xuXG4uZmlsdGVyIFwiYnl0ZXNcIiwgLT5cbiAgKGJ5dGVzLCBwcmVjaXNpb24pIC0+XG4gICAgcmV0dXJuIFwiLVwiICBpZiBpc05hTihwYXJzZUZsb2F0KGJ5dGVzKSkgb3Igbm90IGlzRmluaXRlKGJ5dGVzKVxuICAgIHByZWNpc2lvbiA9IDEgIGlmIHR5cGVvZiBwcmVjaXNpb24gaXMgXCJ1bmRlZmluZWRcIlxuICAgIHVuaXRzID0gWyBcImJ5dGVzXCIsIFwia0JcIiwgXCJNQlwiLCBcIkdCXCIsIFwiVEJcIiwgXCJQQlwiIF1cbiAgICBudW1iZXIgPSBNYXRoLmZsb29yKE1hdGgubG9nKGJ5dGVzKSAvIE1hdGgubG9nKDEwMjQpKVxuICAgIChieXRlcyAvIE1hdGgucG93KDEwMjQsIE1hdGguZmxvb3IobnVtYmVyKSkpLnRvRml4ZWQocHJlY2lzaW9uKSArIFwiIFwiICsgdW5pdHNbbnVtYmVyXVxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBw
 JykuZmlsdGVyKFwiYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkXCIsIGZ1bmN0aW9uKGFuZ3VsYXJNb21lbnRDb25maWcpIHtcbiAgdmFyIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gZnVuY3Rpb24odmFsdWUsIGZvcm1hdCwgZHVyYXRpb25Gb3JtYXQpIHtcbiAgICBpZiAodHlwZW9mIHZhbHVlID09PSBcInVuZGVmaW5lZFwiIHx8IHZhbHVlID09PSBudWxsKSB7XG4gICAgICByZXR1cm4gXCJcIjtcbiAgICB9XG4gICAgcmV0dXJuIG1vbWVudC5kdXJhdGlvbih2YWx1ZSwgZm9ybWF0KS5mb3JtYXQoZHVyYXRpb25Gb3JtYXQsIHtcbiAgICAgIHRyaW06IGZhbHNlXG4gICAgfSk7XG4gIH07XG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlci4kc3RhdGVmdWwgPSBhbmd1bGFyTW9tZW50Q29uZmlnLnN0YXRlZnVsRmlsdGVycztcbiAgcmV0dXJuIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbn0pLmZpbHRlcihcImh1bWFuaXplVGV4dFwiLCBmdW5jdGlvbigpIHtcbiAgcmV0dXJuIGZ1bmN0aW9uKHRleHQpIHtcbiAgICBpZiAodGV4dCkge1xuICAgICAgcmV0dXJuIHRleHQucmVwbGFjZSgvJmd0Oy9nLCBcIj5cIikucmVwbGFjZSgvPGJyXFwvPi9nLCBcIlwiKTtcbiAgICB9IGVsc2Uge1xuICAgICAgcmV0dXJuICcnO1xuICAgIH1cbiAgfTtcbn0pLmZpbHRlcihcImJ5dGVzXCIsIGZ1bmN0aW9uKCkge1xuICByZXR1c
 m4gZnVuY3Rpb24oYnl0ZXMsIHByZWNpc2lvbikge1xuICAgIHZhciBudW1iZXIsIHVuaXRzO1xuICAgIGlmIChpc05hTihwYXJzZUZsb2F0KGJ5dGVzKSkgfHwgIWlzRmluaXRlKGJ5dGVzKSkge1xuICAgICAgcmV0dXJuIFwiLVwiO1xuICAgIH1cbiAgICBpZiAodHlwZW9mIHByZWNpc2lvbiA9PT0gXCJ1bmRlZmluZWRcIikge1xuICAgICAgcHJlY2lzaW9uID0gMTtcbiAgICB9XG4gICAgdW5pdHMgPSBbXCJieXRlc1wiLCBcImtCXCIsIFwiTUJcIiwgXCJHQlwiLCBcIlRCXCIsIFwiUEJcIl07XG4gICAgbnVtYmVyID0gTWF0aC5mbG9vcihNYXRoLmxvZyhieXRlcykgLyBNYXRoLmxvZygxMDI0KSk7XG4gICAgcmV0dXJuIChieXRlcyAvIE1hdGgucG93KDEwMjQsIE1hdGguZmxvb3IobnVtYmVyKSkpLnRvRml4ZWQocHJlY2lzaW9uKSArIFwiIFwiICsgdW5pdHNbbnVtYmVyXTtcbiAgfTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb2
 4gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbi5zZXJ2aWNlICdNYWluU2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRxKSAtPlxuICBAbG9hZENvbmZpZyA9IC0+XG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpXG5cbiAgICAkaHR0cC5nZXQgXCIvY29uZmlnXCJcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb

<TRUNCATED>

[34/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/js/vendor.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/vendor.js b/flink-runtime-web/web-dashboard/web/js/vendor.js
new file mode 100644
index 0000000..52bc81c
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/js/vendor.js
@@ -0,0 +1,81087 @@
+/*!
+ * jQuery JavaScript Library v2.1.4
+ * http://jquery.com/
+ *
+ * Includes Sizzle.js
+ * http://sizzlejs.com/
+ *
+ * Copyright 2005, 2014 jQuery Foundation, Inc. and other contributors
+ * Released under the MIT license
+ * http://jquery.org/license
+ *
+ * Date: 2015-04-28T16:01Z
+ */
+
+(function( global, factory ) {
+
+	if ( typeof module === "object" && typeof module.exports === "object" ) {
+		// For CommonJS and CommonJS-like environments where a proper `window`
+		// is present, execute the factory and get jQuery.
+		// For environments that do not have a `window` with a `document`
+		// (such as Node.js), expose a factory as module.exports.
+		// This accentuates the need for the creation of a real `window`.
+		// e.g. var jQuery = require("jquery")(window);
+		// See ticket #14549 for more info.
+		module.exports = global.document ?
+			factory( global, true ) :
+			function( w ) {
+				if ( !w.document ) {
+					throw new Error( "jQuery requires a window with a document" );
+				}
+				return factory( w );
+			};
+	} else {
+		factory( global );
+	}
+
+// Pass this if window is not defined yet
+}(typeof window !== "undefined" ? window : this, function( window, noGlobal ) {
+
+// Support: Firefox 18+
+// Can't be in strict mode, several libs including ASP.NET trace
+// the stack via arguments.caller.callee and Firefox dies if
+// you try to trace through "use strict" call chains. (#13335)
+//
+
+var arr = [];
+
+var slice = arr.slice;
+
+var concat = arr.concat;
+
+var push = arr.push;
+
+var indexOf = arr.indexOf;
+
+var class2type = {};
+
+var toString = class2type.toString;
+
+var hasOwn = class2type.hasOwnProperty;
+
+var support = {};
+
+
+
+var
+	// Use the correct document accordingly with window argument (sandbox)
+	document = window.document,
+
+	version = "2.1.4",
+
+	// Define a local copy of jQuery
+	jQuery = function( selector, context ) {
+		// The jQuery object is actually just the init constructor 'enhanced'
+		// Need init if jQuery is called (just allow error to be thrown if not included)
+		return new jQuery.fn.init( selector, context );
+	},
+
+	// Support: Android<4.1
+	// Make sure we trim BOM and NBSP
+	rtrim = /^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,
+
+	// Matches dashed string for camelizing
+	rmsPrefix = /^-ms-/,
+	rdashAlpha = /-([\da-z])/gi,
+
+	// Used by jQuery.camelCase as callback to replace()
+	fcamelCase = function( all, letter ) {
+		return letter.toUpperCase();
+	};
+
+jQuery.fn = jQuery.prototype = {
+	// The current version of jQuery being used
+	jquery: version,
+
+	constructor: jQuery,
+
+	// Start with an empty selector
+	selector: "",
+
+	// The default length of a jQuery object is 0
+	length: 0,
+
+	toArray: function() {
+		return slice.call( this );
+	},
+
+	// Get the Nth element in the matched element set OR
+	// Get the whole matched element set as a clean array
+	get: function( num ) {
+		return num != null ?
+
+			// Return just the one element from the set
+			( num < 0 ? this[ num + this.length ] : this[ num ] ) :
+
+			// Return all the elements in a clean array
+			slice.call( this );
+	},
+
+	// Take an array of elements and push it onto the stack
+	// (returning the new matched element set)
+	pushStack: function( elems ) {
+
+		// Build a new jQuery matched element set
+		var ret = jQuery.merge( this.constructor(), elems );
+
+		// Add the old object onto the stack (as a reference)
+		ret.prevObject = this;
+		ret.context = this.context;
+
+		// Return the newly-formed element set
+		return ret;
+	},
+
+	// Execute a callback for every element in the matched set.
+	// (You can seed the arguments with an array of args, but this is
+	// only used internally.)
+	each: function( callback, args ) {
+		return jQuery.each( this, callback, args );
+	},
+
+	map: function( callback ) {
+		return this.pushStack( jQuery.map(this, function( elem, i ) {
+			return callback.call( elem, i, elem );
+		}));
+	},
+
+	slice: function() {
+		return this.pushStack( slice.apply( this, arguments ) );
+	},
+
+	first: function() {
+		return this.eq( 0 );
+	},
+
+	last: function() {
+		return this.eq( -1 );
+	},
+
+	eq: function( i ) {
+		var len = this.length,
+			j = +i + ( i < 0 ? len : 0 );
+		return this.pushStack( j >= 0 && j < len ? [ this[j] ] : [] );
+	},
+
+	end: function() {
+		return this.prevObject || this.constructor(null);
+	},
+
+	// For internal use only.
+	// Behaves like an Array's method, not like a jQuery method.
+	push: push,
+	sort: arr.sort,
+	splice: arr.splice
+};
+
+jQuery.extend = jQuery.fn.extend = function() {
+	var options, name, src, copy, copyIsArray, clone,
+		target = arguments[0] || {},
+		i = 1,
+		length = arguments.length,
+		deep = false;
+
+	// Handle a deep copy situation
+	if ( typeof target === "boolean" ) {
+		deep = target;
+
+		// Skip the boolean and the target
+		target = arguments[ i ] || {};
+		i++;
+	}
+
+	// Handle case when target is a string or something (possible in deep copy)
+	if ( typeof target !== "object" && !jQuery.isFunction(target) ) {
+		target = {};
+	}
+
+	// Extend jQuery itself if only one argument is passed
+	if ( i === length ) {
+		target = this;
+		i--;
+	}
+
+	for ( ; i < length; i++ ) {
+		// Only deal with non-null/undefined values
+		if ( (options = arguments[ i ]) != null ) {
+			// Extend the base object
+			for ( name in options ) {
+				src = target[ name ];
+				copy = options[ name ];
+
+				// Prevent never-ending loop
+				if ( target === copy ) {
+					continue;
+				}
+
+				// Recurse if we're merging plain objects or arrays
+				if ( deep && copy && ( jQuery.isPlainObject(copy) || (copyIsArray = jQuery.isArray(copy)) ) ) {
+					if ( copyIsArray ) {
+						copyIsArray = false;
+						clone = src && jQuery.isArray(src) ? src : [];
+
+					} else {
+						clone = src && jQuery.isPlainObject(src) ? src : {};
+					}
+
+					// Never move original objects, clone them
+					target[ name ] = jQuery.extend( deep, clone, copy );
+
+				// Don't bring in undefined values
+				} else if ( copy !== undefined ) {
+					target[ name ] = copy;
+				}
+			}
+		}
+	}
+
+	// Return the modified object
+	return target;
+};
+
+jQuery.extend({
+	// Unique for each copy of jQuery on the page
+	expando: "jQuery" + ( version + Math.random() ).replace( /\D/g, "" ),
+
+	// Assume jQuery is ready without the ready module
+	isReady: true,
+
+	error: function( msg ) {
+		throw new Error( msg );
+	},
+
+	noop: function() {},
+
+	isFunction: function( obj ) {
+		return jQuery.type(obj) === "function";
+	},
+
+	isArray: Array.isArray,
+
+	isWindow: function( obj ) {
+		return obj != null && obj === obj.window;
+	},
+
+	isNumeric: function( obj ) {
+		// parseFloat NaNs numeric-cast false positives (null|true|false|"")
+		// ...but misinterprets leading-number strings, particularly hex literals ("0x...")
+		// subtraction forces infinities to NaN
+		// adding 1 corrects loss of precision from parseFloat (#15100)
+		return !jQuery.isArray( obj ) && (obj - parseFloat( obj ) + 1) >= 0;
+	},
+
+	isPlainObject: function( obj ) {
+		// Not plain objects:
+		// - Any object or value whose internal [[Class]] property is not "[object Object]"
+		// - DOM nodes
+		// - window
+		if ( jQuery.type( obj ) !== "object" || obj.nodeType || jQuery.isWindow( obj ) ) {
+			return false;
+		}
+
+		if ( obj.constructor &&
+				!hasOwn.call( obj.constructor.prototype, "isPrototypeOf" ) ) {
+			return false;
+		}
+
+		// If the function hasn't returned already, we're confident that
+		// |obj| is a plain object, created by {} or constructed with new Object
+		return true;
+	},
+
+	isEmptyObject: function( obj ) {
+		var name;
+		for ( name in obj ) {
+			return false;
+		}
+		return true;
+	},
+
+	type: function( obj ) {
+		if ( obj == null ) {
+			return obj + "";
+		}
+		// Support: Android<4.0, iOS<6 (functionish RegExp)
+		return typeof obj === "object" || typeof obj === "function" ?
+			class2type[ toString.call(obj) ] || "object" :
+			typeof obj;
+	},
+
+	// Evaluates a script in a global context
+	globalEval: function( code ) {
+		var script,
+			indirect = eval;
+
+		code = jQuery.trim( code );
+
+		if ( code ) {
+			// If the code includes a valid, prologue position
+			// strict mode pragma, execute code by injecting a
+			// script tag into the document.
+			if ( code.indexOf("use strict") === 1 ) {
+				script = document.createElement("script");
+				script.text = code;
+				document.head.appendChild( script ).parentNode.removeChild( script );
+			} else {
+			// Otherwise, avoid the DOM node creation, insertion
+			// and removal by using an indirect global eval
+				indirect( code );
+			}
+		}
+	},
+
+	// Convert dashed to camelCase; used by the css and data modules
+	// Support: IE9-11+
+	// Microsoft forgot to hump their vendor prefix (#9572)
+	camelCase: function( string ) {
+		return string.replace( rmsPrefix, "ms-" ).replace( rdashAlpha, fcamelCase );
+	},
+
+	nodeName: function( elem, name ) {
+		return elem.nodeName && elem.nodeName.toLowerCase() === name.toLowerCase();
+	},
+
+	// args is for internal usage only
+	each: function( obj, callback, args ) {
+		var value,
+			i = 0,
+			length = obj.length,
+			isArray = isArraylike( obj );
+
+		if ( args ) {
+			if ( isArray ) {
+				for ( ; i < length; i++ ) {
+					value = callback.apply( obj[ i ], args );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			} else {
+				for ( i in obj ) {
+					value = callback.apply( obj[ i ], args );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			}
+
+		// A special, fast, case for the most common use of each
+		} else {
+			if ( isArray ) {
+				for ( ; i < length; i++ ) {
+					value = callback.call( obj[ i ], i, obj[ i ] );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			} else {
+				for ( i in obj ) {
+					value = callback.call( obj[ i ], i, obj[ i ] );
+
+					if ( value === false ) {
+						break;
+					}
+				}
+			}
+		}
+
+		return obj;
+	},
+
+	// Support: Android<4.1
+	trim: function( text ) {
+		return text == null ?
+			"" :
+			( text + "" ).replace( rtrim, "" );
+	},
+
+	// results is for internal usage only
+	makeArray: function( arr, results ) {
+		var ret = results || [];
+
+		if ( arr != null ) {
+			if ( isArraylike( Object(arr) ) ) {
+				jQuery.merge( ret,
+					typeof arr === "string" ?
+					[ arr ] : arr
+				);
+			} else {
+				push.call( ret, arr );
+			}
+		}
+
+		return ret;
+	},
+
+	inArray: function( elem, arr, i ) {
+		return arr == null ? -1 : indexOf.call( arr, elem, i );
+	},
+
+	merge: function( first, second ) {
+		var len = +second.length,
+			j = 0,
+			i = first.length;
+
+		for ( ; j < len; j++ ) {
+			first[ i++ ] = second[ j ];
+		}
+
+		first.length = i;
+
+		return first;
+	},
+
+	grep: function( elems, callback, invert ) {
+		var callbackInverse,
+			matches = [],
+			i = 0,
+			length = elems.length,
+			callbackExpect = !invert;
+
+		// Go through the array, only saving the items
+		// that pass the validator function
+		for ( ; i < length; i++ ) {
+			callbackInverse = !callback( elems[ i ], i );
+			if ( callbackInverse !== callbackExpect ) {
+				matches.push( elems[ i ] );
+			}
+		}
+
+		return matches;
+	},
+
+	// arg is for internal usage only
+	map: function( elems, callback, arg ) {
+		var value,
+			i = 0,
+			length = elems.length,
+			isArray = isArraylike( elems ),
+			ret = [];
+
+		// Go through the array, translating each of the items to their new values
+		if ( isArray ) {
+			for ( ; i < length; i++ ) {
+				value = callback( elems[ i ], i, arg );
+
+				if ( value != null ) {
+					ret.push( value );
+				}
+			}
+
+		// Go through every key on the object,
+		} else {
+			for ( i in elems ) {
+				value = callback( elems[ i ], i, arg );
+
+				if ( value != null ) {
+					ret.push( value );
+				}
+			}
+		}
+
+		// Flatten any nested arrays
+		return concat.apply( [], ret );
+	},
+
+	// A global GUID counter for objects
+	guid: 1,
+
+	// Bind a function to a context, optionally partially applying any
+	// arguments.
+	proxy: function( fn, context ) {
+		var tmp, args, proxy;
+
+		if ( typeof context === "string" ) {
+			tmp = fn[ context ];
+			context = fn;
+			fn = tmp;
+		}
+
+		// Quick check to determine if target is callable, in the spec
+		// this throws a TypeError, but we will just return undefined.
+		if ( !jQuery.isFunction( fn ) ) {
+			return undefined;
+		}
+
+		// Simulated bind
+		args = slice.call( arguments, 2 );
+		proxy = function() {
+			return fn.apply( context || this, args.concat( slice.call( arguments ) ) );
+		};
+
+		// Set the guid of unique handler to the same of original handler, so it can be removed
+		proxy.guid = fn.guid = fn.guid || jQuery.guid++;
+
+		return proxy;
+	},
+
+	now: Date.now,
+
+	// jQuery.support is not used in Core but other projects attach their
+	// properties to it so it needs to exist.
+	support: support
+});
+
+// Populate the class2type map
+jQuery.each("Boolean Number String Function Array Date RegExp Object Error".split(" "), function(i, name) {
+	class2type[ "[object " + name + "]" ] = name.toLowerCase();
+});
+
+function isArraylike( obj ) {
+
+	// Support: iOS 8.2 (not reproducible in simulator)
+	// `in` check used to prevent JIT error (gh-2145)
+	// hasOwn isn't used here due to false negatives
+	// regarding Nodelist length in IE
+	var length = "length" in obj && obj.length,
+		type = jQuery.type( obj );
+
+	if ( type === "function" || jQuery.isWindow( obj ) ) {
+		return false;
+	}
+
+	if ( obj.nodeType === 1 && length ) {
+		return true;
+	}
+
+	return type === "array" || length === 0 ||
+		typeof length === "number" && length > 0 && ( length - 1 ) in obj;
+}
+var Sizzle =
+/*!
+ * Sizzle CSS Selector Engine v2.2.0-pre
+ * http://sizzlejs.com/
+ *
+ * Copyright 2008, 2014 jQuery Foundation, Inc. and other contributors
+ * Released under the MIT license
+ * http://jquery.org/license
+ *
+ * Date: 2014-12-16
+ */
+(function( window ) {
+
+var i,
+	support,
+	Expr,
+	getText,
+	isXML,
+	tokenize,
+	compile,
+	select,
+	outermostContext,
+	sortInput,
+	hasDuplicate,
+
+	// Local document vars
+	setDocument,
+	document,
+	docElem,
+	documentIsHTML,
+	rbuggyQSA,
+	rbuggyMatches,
+	matches,
+	contains,
+
+	// Instance-specific data
+	expando = "sizzle" + 1 * new Date(),
+	preferredDoc = window.document,
+	dirruns = 0,
+	done = 0,
+	classCache = createCache(),
+	tokenCache = createCache(),
+	compilerCache = createCache(),
+	sortOrder = function( a, b ) {
+		if ( a === b ) {
+			hasDuplicate = true;
+		}
+		return 0;
+	},
+
+	// General-purpose constants
+	MAX_NEGATIVE = 1 << 31,
+
+	// Instance methods
+	hasOwn = ({}).hasOwnProperty,
+	arr = [],
+	pop = arr.pop,
+	push_native = arr.push,
+	push = arr.push,
+	slice = arr.slice,
+	// Use a stripped-down indexOf as it's faster than native
+	// http://jsperf.com/thor-indexof-vs-for/5
+	indexOf = function( list, elem ) {
+		var i = 0,
+			len = list.length;
+		for ( ; i < len; i++ ) {
+			if ( list[i] === elem ) {
+				return i;
+			}
+		}
+		return -1;
+	},
+
+	booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",
+
+	// Regular expressions
+
+	// Whitespace characters http://www.w3.org/TR/css3-selectors/#whitespace
+	whitespace = "[\\x20\\t\\r\\n\\f]",
+	// http://www.w3.org/TR/css3-syntax/#characters
+	characterEncoding = "(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",
+
+	// Loosely modeled on CSS identifier characters
+	// An unquoted value should be a CSS identifier http://www.w3.org/TR/css3-selectors/#attribute-selectors
+	// Proper syntax: http://www.w3.org/TR/CSS21/syndata.html#value-def-identifier
+	identifier = characterEncoding.replace( "w", "w#" ),
+
+	// Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
+	attributes = "\\[" + whitespace + "*(" + characterEncoding + ")(?:" + whitespace +
+		// Operator (capture 2)
+		"*([*^$|!~]?=)" + whitespace +
+		// "Attribute values must be CSS identifiers [capture 5] or strings [capture 3 or capture 4]"
+		"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + whitespace +
+		"*\\]",
+
+	pseudos = ":(" + characterEncoding + ")(?:\\((" +
+		// To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
+		// 1. quoted (capture 3; capture 4 or capture 5)
+		"('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
+		// 2. simple (capture 6)
+		"((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
+		// 3. anything else (capture 2)
+		".*" +
+		")\\)|)",
+
+	// Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
+	rwhitespace = new RegExp( whitespace + "+", "g" ),
+	rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + whitespace + "+$", "g" ),
+
+	rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
+	rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + "*" ),
+
+	rattributeQuotes = new RegExp( "=" + whitespace + "*([^\\]'\"]*?)" + whitespace + "*\\]", "g" ),
+
+	rpseudo = new RegExp( pseudos ),
+	ridentifier = new RegExp( "^" + identifier + "$" ),
+
+	matchExpr = {
+		"ID": new RegExp( "^#(" + characterEncoding + ")" ),
+		"CLASS": new RegExp( "^\\.(" + characterEncoding + ")" ),
+		"TAG": new RegExp( "^(" + characterEncoding.replace( "w", "w*" ) + ")" ),
+		"ATTR": new RegExp( "^" + attributes ),
+		"PSEUDO": new RegExp( "^" + pseudos ),
+		"CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + whitespace +
+			"*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + whitespace +
+			"*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
+		"bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
+		// For use in libraries implementing .is()
+		// We use this for POS matching in `select`
+		"needsContext": new RegExp( "^" + whitespace + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" +
+			whitespace + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
+	},
+
+	rinputs = /^(?:input|select|textarea|button)$/i,
+	rheader = /^h\d$/i,
+
+	rnative = /^[^{]+\{\s*\[native \w/,
+
+	// Easily-parseable/retrievable ID or TAG or CLASS selectors
+	rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
+
+	rsibling = /[+~]/,
+	rescape = /'|\\/g,
+
+	// CSS escapes http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
+	runescape = new RegExp( "\\\\([\\da-f]{1,6}" + whitespace + "?|(" + whitespace + ")|.)", "ig" ),
+	funescape = function( _, escaped, escapedWhitespace ) {
+		var high = "0x" + escaped - 0x10000;
+		// NaN means non-codepoint
+		// Support: Firefox<24
+		// Workaround erroneous numeric interpretation of +"0x"
+		return high !== high || escapedWhitespace ?
+			escaped :
+			high < 0 ?
+				// BMP codepoint
+				String.fromCharCode( high + 0x10000 ) :
+				// Supplemental Plane codepoint (surrogate pair)
+				String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
+	},
+
+	// Used for iframes
+	// See setDocument()
+	// Removing the function wrapper causes a "Permission Denied"
+	// error in IE
+	unloadHandler = function() {
+		setDocument();
+	};
+
+// Optimize for push.apply( _, NodeList )
+try {
+	push.apply(
+		(arr = slice.call( preferredDoc.childNodes )),
+		preferredDoc.childNodes
+	);
+	// Support: Android<4.0
+	// Detect silently failing push.apply
+	arr[ preferredDoc.childNodes.length ].nodeType;
+} catch ( e ) {
+	push = { apply: arr.length ?
+
+		// Leverage slice if possible
+		function( target, els ) {
+			push_native.apply( target, slice.call(els) );
+		} :
+
+		// Support: IE<9
+		// Otherwise append directly
+		function( target, els ) {
+			var j = target.length,
+				i = 0;
+			// Can't trust NodeList.length
+			while ( (target[j++] = els[i++]) ) {}
+			target.length = j - 1;
+		}
+	};
+}
+
+function Sizzle( selector, context, results, seed ) {
+	var match, elem, m, nodeType,
+		// QSA vars
+		i, groups, old, nid, newContext, newSelector;
+
+	if ( ( context ? context.ownerDocument || context : preferredDoc ) !== document ) {
+		setDocument( context );
+	}
+
+	context = context || document;
+	results = results || [];
+	nodeType = context.nodeType;
+
+	if ( typeof selector !== "string" || !selector ||
+		nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
+
+		return results;
+	}
+
+	if ( !seed && documentIsHTML ) {
+
+		// Try to shortcut find operations when possible (e.g., not under DocumentFragment)
+		if ( nodeType !== 11 && (match = rquickExpr.exec( selector )) ) {
+			// Speed-up: Sizzle("#ID")
+			if ( (m = match[1]) ) {
+				if ( nodeType === 9 ) {
+					elem = context.getElementById( m );
+					// Check parentNode to catch when Blackberry 4.6 returns
+					// nodes that are no longer in the document (jQuery #6963)
+					if ( elem && elem.parentNode ) {
+						// Handle the case where IE, Opera, and Webkit return items
+						// by name instead of ID
+						if ( elem.id === m ) {
+							results.push( elem );
+							return results;
+						}
+					} else {
+						return results;
+					}
+				} else {
+					// Context is not a document
+					if ( context.ownerDocument && (elem = context.ownerDocument.getElementById( m )) &&
+						contains( context, elem ) && elem.id === m ) {
+						results.push( elem );
+						return results;
+					}
+				}
+
+			// Speed-up: Sizzle("TAG")
+			} else if ( match[2] ) {
+				push.apply( results, context.getElementsByTagName( selector ) );
+				return results;
+
+			// Speed-up: Sizzle(".CLASS")
+			} else if ( (m = match[3]) && support.getElementsByClassName ) {
+				push.apply( results, context.getElementsByClassName( m ) );
+				return results;
+			}
+		}
+
+		// QSA path
+		if ( support.qsa && (!rbuggyQSA || !rbuggyQSA.test( selector )) ) {
+			nid = old = expando;
+			newContext = context;
+			newSelector = nodeType !== 1 && selector;
+
+			// qSA works strangely on Element-rooted queries
+			// We can work around this by specifying an extra ID on the root
+			// and working up from there (Thanks to Andrew Dupont for the technique)
+			// IE 8 doesn't work on object elements
+			if ( nodeType === 1 && context.nodeName.toLowerCase() !== "object" ) {
+				groups = tokenize( selector );
+
+				if ( (old = context.getAttribute("id")) ) {
+					nid = old.replace( rescape, "\\$&" );
+				} else {
+					context.setAttribute( "id", nid );
+				}
+				nid = "[id='" + nid + "'] ";
+
+				i = groups.length;
+				while ( i-- ) {
+					groups[i] = nid + toSelector( groups[i] );
+				}
+				newContext = rsibling.test( selector ) && testContext( context.parentNode ) || context;
+				newSelector = groups.join(",");
+			}
+
+			if ( newSelector ) {
+				try {
+					push.apply( results,
+						newContext.querySelectorAll( newSelector )
+					);
+					return results;
+				} catch(qsaError) {
+				} finally {
+					if ( !old ) {
+						context.removeAttribute("id");
+					}
+				}
+			}
+		}
+	}
+
+	// All others
+	return select( selector.replace( rtrim, "$1" ), context, results, seed );
+}
+
+/**
+ * Create key-value caches of limited size
+ * @returns {Function(string, Object)} Returns the Object data after storing it on itself with
+ *	property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
+ *	deleting the oldest entry
+ */
+function createCache() {
+	var keys = [];
+
+	function cache( key, value ) {
+		// Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
+		if ( keys.push( key + " " ) > Expr.cacheLength ) {
+			// Only keep the most recent entries
+			delete cache[ keys.shift() ];
+		}
+		return (cache[ key + " " ] = value);
+	}
+	return cache;
+}
+
+/**
+ * Mark a function for special use by Sizzle
+ * @param {Function} fn The function to mark
+ */
+function markFunction( fn ) {
+	fn[ expando ] = true;
+	return fn;
+}
+
+/**
+ * Support testing using an element
+ * @param {Function} fn Passed the created div and expects a boolean result
+ */
+function assert( fn ) {
+	var div = document.createElement("div");
+
+	try {
+		return !!fn( div );
+	} catch (e) {
+		return false;
+	} finally {
+		// Remove from its parent by default
+		if ( div.parentNode ) {
+			div.parentNode.removeChild( div );
+		}
+		// release memory in IE
+		div = null;
+	}
+}
+
+/**
+ * Adds the same handler for all of the specified attrs
+ * @param {String} attrs Pipe-separated list of attributes
+ * @param {Function} handler The method that will be applied
+ */
+function addHandle( attrs, handler ) {
+	var arr = attrs.split("|"),
+		i = attrs.length;
+
+	while ( i-- ) {
+		Expr.attrHandle[ arr[i] ] = handler;
+	}
+}
+
+/**
+ * Checks document order of two siblings
+ * @param {Element} a
+ * @param {Element} b
+ * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
+ */
+function siblingCheck( a, b ) {
+	var cur = b && a,
+		diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
+			( ~b.sourceIndex || MAX_NEGATIVE ) -
+			( ~a.sourceIndex || MAX_NEGATIVE );
+
+	// Use IE sourceIndex if available on both nodes
+	if ( diff ) {
+		return diff;
+	}
+
+	// Check if b follows a
+	if ( cur ) {
+		while ( (cur = cur.nextSibling) ) {
+			if ( cur === b ) {
+				return -1;
+			}
+		}
+	}
+
+	return a ? 1 : -1;
+}
+
+/**
+ * Returns a function to use in pseudos for input types
+ * @param {String} type
+ */
+function createInputPseudo( type ) {
+	return function( elem ) {
+		var name = elem.nodeName.toLowerCase();
+		return name === "input" && elem.type === type;
+	};
+}
+
+/**
+ * Returns a function to use in pseudos for buttons
+ * @param {String} type
+ */
+function createButtonPseudo( type ) {
+	return function( elem ) {
+		var name = elem.nodeName.toLowerCase();
+		return (name === "input" || name === "button") && elem.type === type;
+	};
+}
+
+/**
+ * Returns a function to use in pseudos for positionals
+ * @param {Function} fn
+ */
+function createPositionalPseudo( fn ) {
+	return markFunction(function( argument ) {
+		argument = +argument;
+		return markFunction(function( seed, matches ) {
+			var j,
+				matchIndexes = fn( [], seed.length, argument ),
+				i = matchIndexes.length;
+
+			// Match elements found at the specified indexes
+			while ( i-- ) {
+				if ( seed[ (j = matchIndexes[i]) ] ) {
+					seed[j] = !(matches[j] = seed[j]);
+				}
+			}
+		});
+	});
+}
+
+/**
+ * Checks a node for validity as a Sizzle context
+ * @param {Element|Object=} context
+ * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
+ */
+function testContext( context ) {
+	return context && typeof context.getElementsByTagName !== "undefined" && context;
+}
+
+// Expose support vars for convenience
+support = Sizzle.support = {};
+
+/**
+ * Detects XML nodes
+ * @param {Element|Object} elem An element or a document
+ * @returns {Boolean} True iff elem is a non-HTML XML node
+ */
+isXML = Sizzle.isXML = function( elem ) {
+	// documentElement is verified for cases where it doesn't yet exist
+	// (such as loading iframes in IE - #4833)
+	var documentElement = elem && (elem.ownerDocument || elem).documentElement;
+	return documentElement ? documentElement.nodeName !== "HTML" : false;
+};
+
+/**
+ * Sets document-related variables once based on the current document
+ * @param {Element|Object} [doc] An element or document object to use to set the document
+ * @returns {Object} Returns the current document
+ */
+setDocument = Sizzle.setDocument = function( node ) {
+	var hasCompare, parent,
+		doc = node ? node.ownerDocument || node : preferredDoc;
+
+	// If no document and documentElement is available, return
+	if ( doc === document || doc.nodeType !== 9 || !doc.documentElement ) {
+		return document;
+	}
+
+	// Set our document
+	document = doc;
+	docElem = doc.documentElement;
+	parent = doc.defaultView;
+
+	// Support: IE>8
+	// If iframe document is assigned to "document" variable and if iframe has been reloaded,
+	// IE will throw "permission denied" error when accessing "document" variable, see jQuery #13936
+	// IE6-8 do not support the defaultView property so parent will be undefined
+	if ( parent && parent !== parent.top ) {
+		// IE11 does not have attachEvent, so all must suffer
+		if ( parent.addEventListener ) {
+			parent.addEventListener( "unload", unloadHandler, false );
+		} else if ( parent.attachEvent ) {
+			parent.attachEvent( "onunload", unloadHandler );
+		}
+	}
+
+	/* Support tests
+	---------------------------------------------------------------------- */
+	documentIsHTML = !isXML( doc );
+
+	/* Attributes
+	---------------------------------------------------------------------- */
+
+	// Support: IE<8
+	// Verify that getAttribute really returns attributes and not properties
+	// (excepting IE8 booleans)
+	support.attributes = assert(function( div ) {
+		div.className = "i";
+		return !div.getAttribute("className");
+	});
+
+	/* getElement(s)By*
+	---------------------------------------------------------------------- */
+
+	// Check if getElementsByTagName("*") returns only elements
+	support.getElementsByTagName = assert(function( div ) {
+		div.appendChild( doc.createComment("") );
+		return !div.getElementsByTagName("*").length;
+	});
+
+	// Support: IE<9
+	support.getElementsByClassName = rnative.test( doc.getElementsByClassName );
+
+	// Support: IE<10
+	// Check if getElementById returns elements by name
+	// The broken getElementById methods don't pick up programatically-set names,
+	// so use a roundabout getElementsByName test
+	support.getById = assert(function( div ) {
+		docElem.appendChild( div ).id = expando;
+		return !doc.getElementsByName || !doc.getElementsByName( expando ).length;
+	});
+
+	// ID find and filter
+	if ( support.getById ) {
+		Expr.find["ID"] = function( id, context ) {
+			if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
+				var m = context.getElementById( id );
+				// Check parentNode to catch when Blackberry 4.6 returns
+				// nodes that are no longer in the document #6963
+				return m && m.parentNode ? [ m ] : [];
+			}
+		};
+		Expr.filter["ID"] = function( id ) {
+			var attrId = id.replace( runescape, funescape );
+			return function( elem ) {
+				return elem.getAttribute("id") === attrId;
+			};
+		};
+	} else {
+		// Support: IE6/7
+		// getElementById is not reliable as a find shortcut
+		delete Expr.find["ID"];
+
+		Expr.filter["ID"] =  function( id ) {
+			var attrId = id.replace( runescape, funescape );
+			return function( elem ) {
+				var node = typeof elem.getAttributeNode !== "undefined" && elem.getAttributeNode("id");
+				return node && node.value === attrId;
+			};
+		};
+	}
+
+	// Tag
+	Expr.find["TAG"] = support.getElementsByTagName ?
+		function( tag, context ) {
+			if ( typeof context.getElementsByTagName !== "undefined" ) {
+				return context.getElementsByTagName( tag );
+
+			// DocumentFragment nodes don't have gEBTN
+			} else if ( support.qsa ) {
+				return context.querySelectorAll( tag );
+			}
+		} :
+
+		function( tag, context ) {
+			var elem,
+				tmp = [],
+				i = 0,
+				// By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
+				results = context.getElementsByTagName( tag );
+
+			// Filter out possible comments
+			if ( tag === "*" ) {
+				while ( (elem = results[i++]) ) {
+					if ( elem.nodeType === 1 ) {
+						tmp.push( elem );
+					}
+				}
+
+				return tmp;
+			}
+			return results;
+		};
+
+	// Class
+	Expr.find["CLASS"] = support.getElementsByClassName && function( className, context ) {
+		if ( documentIsHTML ) {
+			return context.getElementsByClassName( className );
+		}
+	};
+
+	/* QSA/matchesSelector
+	---------------------------------------------------------------------- */
+
+	// QSA and matchesSelector support
+
+	// matchesSelector(:active) reports false when true (IE9/Opera 11.5)
+	rbuggyMatches = [];
+
+	// qSa(:focus) reports false when true (Chrome 21)
+	// We allow this because of a bug in IE8/9 that throws an error
+	// whenever `document.activeElement` is accessed on an iframe
+	// So, we allow :focus to pass through QSA all the time to avoid the IE error
+	// See http://bugs.jquery.com/ticket/13378
+	rbuggyQSA = [];
+
+	if ( (support.qsa = rnative.test( doc.querySelectorAll )) ) {
+		// Build QSA regex
+		// Regex strategy adopted from Diego Perini
+		assert(function( div ) {
+			// Select is set to empty string on purpose
+			// This is to test IE's treatment of not explicitly
+			// setting a boolean content attribute,
+			// since its presence should be enough
+			// http://bugs.jquery.com/ticket/12359
+			docElem.appendChild( div ).innerHTML = "<a id='" + expando + "'></a>" +
+				"<select id='" + expando + "-\f]' msallowcapture=''>" +
+				"<option selected=''></option></select>";
+
+			// Support: IE8, Opera 11-12.16
+			// Nothing should be selected when empty strings follow ^= or $= or *=
+			// The test attribute must be unknown in Opera but "safe" for WinRT
+			// http://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
+			if ( div.querySelectorAll("[msallowcapture^='']").length ) {
+				rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
+			}
+
+			// Support: IE8
+			// Boolean attributes and "value" are not treated correctly
+			if ( !div.querySelectorAll("[selected]").length ) {
+				rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
+			}
+
+			// Support: Chrome<29, Android<4.2+, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.7+
+			if ( !div.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
+				rbuggyQSA.push("~=");
+			}
+
+			// Webkit/Opera - :checked should return selected option elements
+			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+			// IE8 throws error here and will not see later tests
+			if ( !div.querySelectorAll(":checked").length ) {
+				rbuggyQSA.push(":checked");
+			}
+
+			// Support: Safari 8+, iOS 8+
+			// https://bugs.webkit.org/show_bug.cgi?id=136851
+			// In-page `selector#id sibing-combinator selector` fails
+			if ( !div.querySelectorAll( "a#" + expando + "+*" ).length ) {
+				rbuggyQSA.push(".#.+[+~]");
+			}
+		});
+
+		assert(function( div ) {
+			// Support: Windows 8 Native Apps
+			// The type and name attributes are restricted during .innerHTML assignment
+			var input = doc.createElement("input");
+			input.setAttribute( "type", "hidden" );
+			div.appendChild( input ).setAttribute( "name", "D" );
+
+			// Support: IE8
+			// Enforce case-sensitivity of name attribute
+			if ( div.querySelectorAll("[name=d]").length ) {
+				rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
+			}
+
+			// FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
+			// IE8 throws error here and will not see later tests
+			if ( !div.querySelectorAll(":enabled").length ) {
+				rbuggyQSA.push( ":enabled", ":disabled" );
+			}
+
+			// Opera 10-11 does not throw on post-comma invalid pseudos
+			div.querySelectorAll("*,:x");
+			rbuggyQSA.push(",.*:");
+		});
+	}
+
+	if ( (support.matchesSelector = rnative.test( (matches = docElem.matches ||
+		docElem.webkitMatchesSelector ||
+		docElem.mozMatchesSelector ||
+		docElem.oMatchesSelector ||
+		docElem.msMatchesSelector) )) ) {
+
+		assert(function( div ) {
+			// Check to see if it's possible to do matchesSelector
+			// on a disconnected node (IE 9)
+			support.disconnectedMatch = matches.call( div, "div" );
+
+			// This should fail with an exception
+			// Gecko does not error, returns false instead
+			matches.call( div, "[s!='']:x" );
+			rbuggyMatches.push( "!=", pseudos );
+		});
+	}
+
+	rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join("|") );
+	rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join("|") );
+
+	/* Contains
+	---------------------------------------------------------------------- */
+	hasCompare = rnative.test( docElem.compareDocumentPosition );
+
+	// Element contains another
+	// Purposefully does not implement inclusive descendent
+	// As in, an element does not contain itself
+	contains = hasCompare || rnative.test( docElem.contains ) ?
+		function( a, b ) {
+			var adown = a.nodeType === 9 ? a.documentElement : a,
+				bup = b && b.parentNode;
+			return a === bup || !!( bup && bup.nodeType === 1 && (
+				adown.contains ?
+					adown.contains( bup ) :
+					a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
+			));
+		} :
+		function( a, b ) {
+			if ( b ) {
+				while ( (b = b.parentNode) ) {
+					if ( b === a ) {
+						return true;
+					}
+				}
+			}
+			return false;
+		};
+
+	/* Sorting
+	---------------------------------------------------------------------- */
+
+	// Document order sorting
+	sortOrder = hasCompare ?
+	function( a, b ) {
+
+		// Flag for duplicate removal
+		if ( a === b ) {
+			hasDuplicate = true;
+			return 0;
+		}
+
+		// Sort on method existence if only one input has compareDocumentPosition
+		var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
+		if ( compare ) {
+			return compare;
+		}
+
+		// Calculate position if both inputs belong to the same document
+		compare = ( a.ownerDocument || a ) === ( b.ownerDocument || b ) ?
+			a.compareDocumentPosition( b ) :
+
+			// Otherwise we know they are disconnected
+			1;
+
+		// Disconnected nodes
+		if ( compare & 1 ||
+			(!support.sortDetached && b.compareDocumentPosition( a ) === compare) ) {
+
+			// Choose the first element that is related to our preferred document
+			if ( a === doc || a.ownerDocument === preferredDoc && contains(preferredDoc, a) ) {
+				return -1;
+			}
+			if ( b === doc || b.ownerDocument === preferredDoc && contains(preferredDoc, b) ) {
+				return 1;
+			}
+
+			// Maintain original order
+			return sortInput ?
+				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+				0;
+		}
+
+		return compare & 4 ? -1 : 1;
+	} :
+	function( a, b ) {
+		// Exit early if the nodes are identical
+		if ( a === b ) {
+			hasDuplicate = true;
+			return 0;
+		}
+
+		var cur,
+			i = 0,
+			aup = a.parentNode,
+			bup = b.parentNode,
+			ap = [ a ],
+			bp = [ b ];
+
+		// Parentless nodes are either documents or disconnected
+		if ( !aup || !bup ) {
+			return a === doc ? -1 :
+				b === doc ? 1 :
+				aup ? -1 :
+				bup ? 1 :
+				sortInput ?
+				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+				0;
+
+		// If the nodes are siblings, we can do a quick check
+		} else if ( aup === bup ) {
+			return siblingCheck( a, b );
+		}
+
+		// Otherwise we need full lists of their ancestors for comparison
+		cur = a;
+		while ( (cur = cur.parentNode) ) {
+			ap.unshift( cur );
+		}
+		cur = b;
+		while ( (cur = cur.parentNode) ) {
+			bp.unshift( cur );
+		}
+
+		// Walk down the tree looking for a discrepancy
+		while ( ap[i] === bp[i] ) {
+			i++;
+		}
+
+		return i ?
+			// Do a sibling check if the nodes have a common ancestor
+			siblingCheck( ap[i], bp[i] ) :
+
+			// Otherwise nodes in our document sort first
+			ap[i] === preferredDoc ? -1 :
+			bp[i] === preferredDoc ? 1 :
+			0;
+	};
+
+	return doc;
+};
+
+Sizzle.matches = function( expr, elements ) {
+	return Sizzle( expr, null, null, elements );
+};
+
+Sizzle.matchesSelector = function( elem, expr ) {
+	// Set document vars if needed
+	if ( ( elem.ownerDocument || elem ) !== document ) {
+		setDocument( elem );
+	}
+
+	// Make sure that attribute selectors are quoted
+	expr = expr.replace( rattributeQuotes, "='$1']" );
+
+	if ( support.matchesSelector && documentIsHTML &&
+		( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
+		( !rbuggyQSA     || !rbuggyQSA.test( expr ) ) ) {
+
+		try {
+			var ret = matches.call( elem, expr );
+
+			// IE 9's matchesSelector returns false on disconnected nodes
+			if ( ret || support.disconnectedMatch ||
+					// As well, disconnected nodes are said to be in a document
+					// fragment in IE 9
+					elem.document && elem.document.nodeType !== 11 ) {
+				return ret;
+			}
+		} catch (e) {}
+	}
+
+	return Sizzle( expr, document, null, [ elem ] ).length > 0;
+};
+
+Sizzle.contains = function( context, elem ) {
+	// Set document vars if needed
+	if ( ( context.ownerDocument || context ) !== document ) {
+		setDocument( context );
+	}
+	return contains( context, elem );
+};
+
+Sizzle.attr = function( elem, name ) {
+	// Set document vars if needed
+	if ( ( elem.ownerDocument || elem ) !== document ) {
+		setDocument( elem );
+	}
+
+	var fn = Expr.attrHandle[ name.toLowerCase() ],
+		// Don't get fooled by Object.prototype properties (jQuery #13807)
+		val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
+			fn( elem, name, !documentIsHTML ) :
+			undefined;
+
+	return val !== undefined ?
+		val :
+		support.attributes || !documentIsHTML ?
+			elem.getAttribute( name ) :
+			(val = elem.getAttributeNode(name)) && val.specified ?
+				val.value :
+				null;
+};
+
+Sizzle.error = function( msg ) {
+	throw new Error( "Syntax error, unrecognized expression: " + msg );
+};
+
+/**
+ * Document sorting and removing duplicates
+ * @param {ArrayLike} results
+ */
+Sizzle.uniqueSort = function( results ) {
+	var elem,
+		duplicates = [],
+		j = 0,
+		i = 0;
+
+	// Unless we *know* we can detect duplicates, assume their presence
+	hasDuplicate = !support.detectDuplicates;
+	sortInput = !support.sortStable && results.slice( 0 );
+	results.sort( sortOrder );
+
+	if ( hasDuplicate ) {
+		while ( (elem = results[i++]) ) {
+			if ( elem === results[ i ] ) {
+				j = duplicates.push( i );
+			}
+		}
+		while ( j-- ) {
+			results.splice( duplicates[ j ], 1 );
+		}
+	}
+
+	// Clear input after sorting to release objects
+	// See https://github.com/jquery/sizzle/pull/225
+	sortInput = null;
+
+	return results;
+};
+
+/**
+ * Utility function for retrieving the text value of an array of DOM nodes
+ * @param {Array|Element} elem
+ */
+getText = Sizzle.getText = function( elem ) {
+	var node,
+		ret = "",
+		i = 0,
+		nodeType = elem.nodeType;
+
+	if ( !nodeType ) {
+		// If no nodeType, this is expected to be an array
+		while ( (node = elem[i++]) ) {
+			// Do not traverse comment nodes
+			ret += getText( node );
+		}
+	} else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
+		// Use textContent for elements
+		// innerText usage removed for consistency of new lines (jQuery #11153)
+		if ( typeof elem.textContent === "string" ) {
+			return elem.textContent;
+		} else {
+			// Traverse its children
+			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+				ret += getText( elem );
+			}
+		}
+	} else if ( nodeType === 3 || nodeType === 4 ) {
+		return elem.nodeValue;
+	}
+	// Do not include comment or processing instruction nodes
+
+	return ret;
+};
+
+Expr = Sizzle.selectors = {
+
+	// Can be adjusted by the user
+	cacheLength: 50,
+
+	createPseudo: markFunction,
+
+	match: matchExpr,
+
+	attrHandle: {},
+
+	find: {},
+
+	relative: {
+		">": { dir: "parentNode", first: true },
+		" ": { dir: "parentNode" },
+		"+": { dir: "previousSibling", first: true },
+		"~": { dir: "previousSibling" }
+	},
+
+	preFilter: {
+		"ATTR": function( match ) {
+			match[1] = match[1].replace( runescape, funescape );
+
+			// Move the given value to match[3] whether quoted or unquoted
+			match[3] = ( match[3] || match[4] || match[5] || "" ).replace( runescape, funescape );
+
+			if ( match[2] === "~=" ) {
+				match[3] = " " + match[3] + " ";
+			}
+
+			return match.slice( 0, 4 );
+		},
+
+		"CHILD": function( match ) {
+			/* matches from matchExpr["CHILD"]
+				1 type (only|nth|...)
+				2 what (child|of-type)
+				3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
+				4 xn-component of xn+y argument ([+-]?\d*n|)
+				5 sign of xn-component
+				6 x of xn-component
+				7 sign of y-component
+				8 y of y-component
+			*/
+			match[1] = match[1].toLowerCase();
+
+			if ( match[1].slice( 0, 3 ) === "nth" ) {
+				// nth-* requires argument
+				if ( !match[3] ) {
+					Sizzle.error( match[0] );
+				}
+
+				// numeric x and y parameters for Expr.filter.CHILD
+				// remember that false/true cast respectively to 0/1
+				match[4] = +( match[4] ? match[5] + (match[6] || 1) : 2 * ( match[3] === "even" || match[3] === "odd" ) );
+				match[5] = +( ( match[7] + match[8] ) || match[3] === "odd" );
+
+			// other types prohibit arguments
+			} else if ( match[3] ) {
+				Sizzle.error( match[0] );
+			}
+
+			return match;
+		},
+
+		"PSEUDO": function( match ) {
+			var excess,
+				unquoted = !match[6] && match[2];
+
+			if ( matchExpr["CHILD"].test( match[0] ) ) {
+				return null;
+			}
+
+			// Accept quoted arguments as-is
+			if ( match[3] ) {
+				match[2] = match[4] || match[5] || "";
+
+			// Strip excess characters from unquoted arguments
+			} else if ( unquoted && rpseudo.test( unquoted ) &&
+				// Get excess from tokenize (recursively)
+				(excess = tokenize( unquoted, true )) &&
+				// advance to the next closing parenthesis
+				(excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length) ) {
+
+				// excess is a negative index
+				match[0] = match[0].slice( 0, excess );
+				match[2] = unquoted.slice( 0, excess );
+			}
+
+			// Return only captures needed by the pseudo filter method (type and argument)
+			return match.slice( 0, 3 );
+		}
+	},
+
+	filter: {
+
+		"TAG": function( nodeNameSelector ) {
+			var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
+			return nodeNameSelector === "*" ?
+				function() { return true; } :
+				function( elem ) {
+					return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
+				};
+		},
+
+		"CLASS": function( className ) {
+			var pattern = classCache[ className + " " ];
+
+			return pattern ||
+				(pattern = new RegExp( "(^|" + whitespace + ")" + className + "(" + whitespace + "|$)" )) &&
+				classCache( className, function( elem ) {
+					return pattern.test( typeof elem.className === "string" && elem.className || typeof elem.getAttribute !== "undefined" && elem.getAttribute("class") || "" );
+				});
+		},
+
+		"ATTR": function( name, operator, check ) {
+			return function( elem ) {
+				var result = Sizzle.attr( elem, name );
+
+				if ( result == null ) {
+					return operator === "!=";
+				}
+				if ( !operator ) {
+					return true;
+				}
+
+				result += "";
+
+				return operator === "=" ? result === check :
+					operator === "!=" ? result !== check :
+					operator === "^=" ? check && result.indexOf( check ) === 0 :
+					operator === "*=" ? check && result.indexOf( check ) > -1 :
+					operator === "$=" ? check && result.slice( -check.length ) === check :
+					operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
+					operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
+					false;
+			};
+		},
+
+		"CHILD": function( type, what, argument, first, last ) {
+			var simple = type.slice( 0, 3 ) !== "nth",
+				forward = type.slice( -4 ) !== "last",
+				ofType = what === "of-type";
+
+			return first === 1 && last === 0 ?
+
+				// Shortcut for :nth-*(n)
+				function( elem ) {
+					return !!elem.parentNode;
+				} :
+
+				function( elem, context, xml ) {
+					var cache, outerCache, node, diff, nodeIndex, start,
+						dir = simple !== forward ? "nextSibling" : "previousSibling",
+						parent = elem.parentNode,
+						name = ofType && elem.nodeName.toLowerCase(),
+						useCache = !xml && !ofType;
+
+					if ( parent ) {
+
+						// :(first|last|only)-(child|of-type)
+						if ( simple ) {
+							while ( dir ) {
+								node = elem;
+								while ( (node = node[ dir ]) ) {
+									if ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) {
+										return false;
+									}
+								}
+								// Reverse direction for :only-* (if we haven't yet done so)
+								start = dir = type === "only" && !start && "nextSibling";
+							}
+							return true;
+						}
+
+						start = [ forward ? parent.firstChild : parent.lastChild ];
+
+						// non-xml :nth-child(...) stores cache data on `parent`
+						if ( forward && useCache ) {
+							// Seek `elem` from a previously-cached index
+							outerCache = parent[ expando ] || (parent[ expando ] = {});
+							cache = outerCache[ type ] || [];
+							nodeIndex = cache[0] === dirruns && cache[1];
+							diff = cache[0] === dirruns && cache[2];
+							node = nodeIndex && parent.childNodes[ nodeIndex ];
+
+							while ( (node = ++nodeIndex && node && node[ dir ] ||
+
+								// Fallback to seeking `elem` from the start
+								(diff = nodeIndex = 0) || start.pop()) ) {
+
+								// When found, cache indexes on `parent` and break
+								if ( node.nodeType === 1 && ++diff && node === elem ) {
+									outerCache[ type ] = [ dirruns, nodeIndex, diff ];
+									break;
+								}
+							}
+
+						// Use previously-cached element index if available
+						} else if ( useCache && (cache = (elem[ expando ] || (elem[ expando ] = {}))[ type ]) && cache[0] === dirruns ) {
+							diff = cache[1];
+
+						// xml :nth-child(...) or :nth-last-child(...) or :nth(-last)?-of-type(...)
+						} else {
+							// Use the same loop as above to seek `elem` from the start
+							while ( (node = ++nodeIndex && node && node[ dir ] ||
+								(diff = nodeIndex = 0) || start.pop()) ) {
+
+								if ( ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) && ++diff ) {
+									// Cache the index of each encountered element
+									if ( useCache ) {
+										(node[ expando ] || (node[ expando ] = {}))[ type ] = [ dirruns, diff ];
+									}
+
+									if ( node === elem ) {
+										break;
+									}
+								}
+							}
+						}
+
+						// Incorporate the offset, then check against cycle size
+						diff -= last;
+						return diff === first || ( diff % first === 0 && diff / first >= 0 );
+					}
+				};
+		},
+
+		"PSEUDO": function( pseudo, argument ) {
+			// pseudo-class names are case-insensitive
+			// http://www.w3.org/TR/selectors/#pseudo-classes
+			// Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
+			// Remember that setFilters inherits from pseudos
+			var args,
+				fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
+					Sizzle.error( "unsupported pseudo: " + pseudo );
+
+			// The user may use createPseudo to indicate that
+			// arguments are needed to create the filter function
+			// just as Sizzle does
+			if ( fn[ expando ] ) {
+				return fn( argument );
+			}
+
+			// But maintain support for old signatures
+			if ( fn.length > 1 ) {
+				args = [ pseudo, pseudo, "", argument ];
+				return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
+					markFunction(function( seed, matches ) {
+						var idx,
+							matched = fn( seed, argument ),
+							i = matched.length;
+						while ( i-- ) {
+							idx = indexOf( seed, matched[i] );
+							seed[ idx ] = !( matches[ idx ] = matched[i] );
+						}
+					}) :
+					function( elem ) {
+						return fn( elem, 0, args );
+					};
+			}
+
+			return fn;
+		}
+	},
+
+	pseudos: {
+		// Potentially complex pseudos
+		"not": markFunction(function( selector ) {
+			// Trim the selector passed to compile
+			// to avoid treating leading and trailing
+			// spaces as combinators
+			var input = [],
+				results = [],
+				matcher = compile( selector.replace( rtrim, "$1" ) );
+
+			return matcher[ expando ] ?
+				markFunction(function( seed, matches, context, xml ) {
+					var elem,
+						unmatched = matcher( seed, null, xml, [] ),
+						i = seed.length;
+
+					// Match elements unmatched by `matcher`
+					while ( i-- ) {
+						if ( (elem = unmatched[i]) ) {
+							seed[i] = !(matches[i] = elem);
+						}
+					}
+				}) :
+				function( elem, context, xml ) {
+					input[0] = elem;
+					matcher( input, null, xml, results );
+					// Don't keep the element (issue #299)
+					input[0] = null;
+					return !results.pop();
+				};
+		}),
+
+		"has": markFunction(function( selector ) {
+			return function( elem ) {
+				return Sizzle( selector, elem ).length > 0;
+			};
+		}),
+
+		"contains": markFunction(function( text ) {
+			text = text.replace( runescape, funescape );
+			return function( elem ) {
+				return ( elem.textContent || elem.innerText || getText( elem ) ).indexOf( text ) > -1;
+			};
+		}),
+
+		// "Whether an element is represented by a :lang() selector
+		// is based solely on the element's language value
+		// being equal to the identifier C,
+		// or beginning with the identifier C immediately followed by "-".
+		// The matching of C against the element's language value is performed case-insensitively.
+		// The identifier C does not have to be a valid language name."
+		// http://www.w3.org/TR/selectors/#lang-pseudo
+		"lang": markFunction( function( lang ) {
+			// lang value must be a valid identifier
+			if ( !ridentifier.test(lang || "") ) {
+				Sizzle.error( "unsupported lang: " + lang );
+			}
+			lang = lang.replace( runescape, funescape ).toLowerCase();
+			return function( elem ) {
+				var elemLang;
+				do {
+					if ( (elemLang = documentIsHTML ?
+						elem.lang :
+						elem.getAttribute("xml:lang") || elem.getAttribute("lang")) ) {
+
+						elemLang = elemLang.toLowerCase();
+						return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
+					}
+				} while ( (elem = elem.parentNode) && elem.nodeType === 1 );
+				return false;
+			};
+		}),
+
+		// Miscellaneous
+		"target": function( elem ) {
+			var hash = window.location && window.location.hash;
+			return hash && hash.slice( 1 ) === elem.id;
+		},
+
+		"root": function( elem ) {
+			return elem === docElem;
+		},
+
+		"focus": function( elem ) {
+			return elem === document.activeElement && (!document.hasFocus || document.hasFocus()) && !!(elem.type || elem.href || ~elem.tabIndex);
+		},
+
+		// Boolean properties
+		"enabled": function( elem ) {
+			return elem.disabled === false;
+		},
+
+		"disabled": function( elem ) {
+			return elem.disabled === true;
+		},
+
+		"checked": function( elem ) {
+			// In CSS3, :checked should return both checked and selected elements
+			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+			var nodeName = elem.nodeName.toLowerCase();
+			return (nodeName === "input" && !!elem.checked) || (nodeName === "option" && !!elem.selected);
+		},
+
+		"selected": function( elem ) {
+			// Accessing this property makes selected-by-default
+			// options in Safari work properly
+			if ( elem.parentNode ) {
+				elem.parentNode.selectedIndex;
+			}
+
+			return elem.selected === true;
+		},
+
+		// Contents
+		"empty": function( elem ) {
+			// http://www.w3.org/TR/selectors/#empty-pseudo
+			// :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
+			//   but not by others (comment: 8; processing instruction: 7; etc.)
+			// nodeType < 6 works because attributes (2) do not appear as children
+			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+				if ( elem.nodeType < 6 ) {
+					return false;
+				}
+			}
+			return true;
+		},
+
+		"parent": function( elem ) {
+			return !Expr.pseudos["empty"]( elem );
+		},
+
+		// Element/input types
+		"header": function( elem ) {
+			return rheader.test( elem.nodeName );
+		},
+
+		"input": function( elem ) {
+			return rinputs.test( elem.nodeName );
+		},
+
+		"button": function( elem ) {
+			var name = elem.nodeName.toLowerCase();
+			return name === "input" && elem.type === "button" || name === "button";
+		},
+
+		"text": function( elem ) {
+			var attr;
+			return elem.nodeName.toLowerCase() === "input" &&
+				elem.type === "text" &&
+
+				// Support: IE<8
+				// New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
+				( (attr = elem.getAttribute("type")) == null || attr.toLowerCase() === "text" );
+		},
+
+		// Position-in-collection
+		"first": createPositionalPseudo(function() {
+			return [ 0 ];
+		}),
+
+		"last": createPositionalPseudo(function( matchIndexes, length ) {
+			return [ length - 1 ];
+		}),
+
+		"eq": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			return [ argument < 0 ? argument + length : argument ];
+		}),
+
+		"even": createPositionalPseudo(function( matchIndexes, length ) {
+			var i = 0;
+			for ( ; i < length; i += 2 ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"odd": createPositionalPseudo(function( matchIndexes, length ) {
+			var i = 1;
+			for ( ; i < length; i += 2 ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"lt": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			var i = argument < 0 ? argument + length : argument;
+			for ( ; --i >= 0; ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		}),
+
+		"gt": createPositionalPseudo(function( matchIndexes, length, argument ) {
+			var i = argument < 0 ? argument + length : argument;
+			for ( ; ++i < length; ) {
+				matchIndexes.push( i );
+			}
+			return matchIndexes;
+		})
+	}
+};
+
+Expr.pseudos["nth"] = Expr.pseudos["eq"];
+
+// Add button/input type pseudos
+for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
+	Expr.pseudos[ i ] = createInputPseudo( i );
+}
+for ( i in { submit: true, reset: true } ) {
+	Expr.pseudos[ i ] = createButtonPseudo( i );
+}
+
+// Easy API for creating new setFilters
+function setFilters() {}
+setFilters.prototype = Expr.filters = Expr.pseudos;
+Expr.setFilters = new setFilters();
+
+tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
+	var matched, match, tokens, type,
+		soFar, groups, preFilters,
+		cached = tokenCache[ selector + " " ];
+
+	if ( cached ) {
+		return parseOnly ? 0 : cached.slice( 0 );
+	}
+
+	soFar = selector;
+	groups = [];
+	preFilters = Expr.preFilter;
+
+	while ( soFar ) {
+
+		// Comma and first run
+		if ( !matched || (match = rcomma.exec( soFar )) ) {
+			if ( match ) {
+				// Don't consume trailing commas as valid
+				soFar = soFar.slice( match[0].length ) || soFar;
+			}
+			groups.push( (tokens = []) );
+		}
+
+		matched = false;
+
+		// Combinators
+		if ( (match = rcombinators.exec( soFar )) ) {
+			matched = match.shift();
+			tokens.push({
+				value: matched,
+				// Cast descendant combinators to space
+				type: match[0].replace( rtrim, " " )
+			});
+			soFar = soFar.slice( matched.length );
+		}
+
+		// Filters
+		for ( type in Expr.filter ) {
+			if ( (match = matchExpr[ type ].exec( soFar )) && (!preFilters[ type ] ||
+				(match = preFilters[ type ]( match ))) ) {
+				matched = match.shift();
+				tokens.push({
+					value: matched,
+					type: type,
+					matches: match
+				});
+				soFar = soFar.slice( matched.length );
+			}
+		}
+
+		if ( !matched ) {
+			break;
+		}
+	}
+
+	// Return the length of the invalid excess
+	// if we're just parsing
+	// Otherwise, throw an error or return tokens
+	return parseOnly ?
+		soFar.length :
+		soFar ?
+			Sizzle.error( selector ) :
+			// Cache the tokens
+			tokenCache( selector, groups ).slice( 0 );
+};
+
+function toSelector( tokens ) {
+	var i = 0,
+		len = tokens.length,
+		selector = "";
+	for ( ; i < len; i++ ) {
+		selector += tokens[i].value;
+	}
+	return selector;
+}
+
+function addCombinator( matcher, combinator, base ) {
+	var dir = combinator.dir,
+		checkNonElements = base && dir === "parentNode",
+		doneName = done++;
+
+	return combinator.first ?
+		// Check against closest ancestor/preceding element
+		function( elem, context, xml ) {
+			while ( (elem = elem[ dir ]) ) {
+				if ( elem.nodeType === 1 || checkNonElements ) {
+					return matcher( elem, context, xml );
+				}
+			}
+		} :
+
+		// Check against all ancestor/preceding elements
+		function( elem, context, xml ) {
+			var oldCache, outerCache,
+				newCache = [ dirruns, doneName ];
+
+			// We can't set arbitrary data on XML nodes, so they don't benefit from dir caching
+			if ( xml ) {
+				while ( (elem = elem[ dir ]) ) {
+					if ( elem.nodeType === 1 || checkNonElements ) {
+						if ( matcher( elem, context, xml ) ) {
+							return true;
+						}
+					}
+				}
+			} else {
+				while ( (elem = elem[ dir ]) ) {
+					if ( elem.nodeType === 1 || checkNonElements ) {
+						outerCache = elem[ expando ] || (elem[ expando ] = {});
+						if ( (oldCache = outerCache[ dir ]) &&
+							oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
+
+							// Assign to newCache so results back-propagate to previous elements
+							return (newCache[ 2 ] = oldCache[ 2 ]);
+						} else {
+							// Reuse newcache so results back-propagate to previous elements
+							outerCache[ dir ] = newCache;
+
+							// A match means we're done; a fail means we have to keep checking
+							if ( (newCache[ 2 ] = matcher( elem, context, xml )) ) {
+								return true;
+							}
+						}
+					}
+				}
+			}
+		};
+}
+
+function elementMatcher( matchers ) {
+	return matchers.length > 1 ?
+		function( elem, context, xml ) {
+			var i = matchers.length;
+			while ( i-- ) {
+				if ( !matchers[i]( elem, context, xml ) ) {
+					return false;
+				}
+			}
+			return true;
+		} :
+		matchers[0];
+}
+
+function multipleContexts( selector, contexts, results ) {
+	var i = 0,
+		len = contexts.length;
+	for ( ; i < len; i++ ) {
+		Sizzle( selector, contexts[i], results );
+	}
+	return results;
+}
+
+function condense( unmatched, map, filter, context, xml ) {
+	var elem,
+		newUnmatched = [],
+		i = 0,
+		len = unmatched.length,
+		mapped = map != null;
+
+	for ( ; i < len; i++ ) {
+		if ( (elem = unmatched[i]) ) {
+			if ( !filter || filter( elem, context, xml ) ) {
+				newUnmatched.push( elem );
+				if ( mapped ) {
+					map.push( i );
+				}
+			}
+		}
+	}
+
+	return newUnmatched;
+}
+
+function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
+	if ( postFilter && !postFilter[ expando ] ) {
+		postFilter = setMatcher( postFilter );
+	}
+	if ( postFinder && !postFinder[ expando ] ) {
+		postFinder = setMatcher( postFinder, postSelector );
+	}
+	return markFunction(function( seed, results, context, xml ) {
+		var temp, i, elem,
+			preMap = [],
+			postMap = [],
+			preexisting = results.length,
+
+			// Get initial elements from seed or context
+			elems = seed || multipleContexts( selector || "*", context.nodeType ? [ context ] : context, [] ),
+
+			// Prefilter to get matcher input, preserving a map for seed-results synchronization
+			matcherIn = preFilter && ( seed || !selector ) ?
+				condense( elems, preMap, preFilter, context, xml ) :
+				elems,
+
+			matcherOut = matcher ?
+				// If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
+				postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
+
+					// ...intermediate processing is necessary
+					[] :
+
+					// ...otherwise use results directly
+					results :
+				matcherIn;
+
+		// Find primary matches
+		if ( matcher ) {
+			matcher( matcherIn, matcherOut, context, xml );
+		}
+
+		// Apply postFilter
+		if ( postFilter ) {
+			temp = condense( matcherOut, postMap );
+			postFilter( temp, [], context, xml );
+
+			// Un-match failing elements by moving them back to matcherIn
+			i = temp.length;
+			while ( i-- ) {
+				if ( (elem = temp[i]) ) {
+					matcherOut[ postMap[i] ] = !(matcherIn[ postMap[i] ] = elem);
+				}
+			}
+		}
+
+		if ( seed ) {
+			if ( postFinder || preFilter ) {
+				if ( postFinder ) {
+					// Get the final matcherOut by condensing this intermediate into postFinder contexts
+					temp = [];
+					i = matcherOut.length;
+					while ( i-- ) {
+						if ( (elem = matcherOut[i]) ) {
+							// Restore matcherIn since elem is not yet a final match
+							temp.push( (matcherIn[i] = elem) );
+						}
+					}
+					postFinder( null, (matcherOut = []), temp, xml );
+				}
+
+				// Move matched elements from seed to results to keep them synchronized
+				i = matcherOut.length;
+				while ( i-- ) {
+					if ( (elem = matcherOut[i]) &&
+						(temp = postFinder ? indexOf( seed, elem ) : preMap[i]) > -1 ) {
+
+						seed[temp] = !(results[temp] = elem);
+					}
+				}
+			}
+
+		// Add elements to results, through postFinder if defined
+		} else {
+			matcherOut = condense(
+				matcherOut === results ?
+					matcherOut.splice( preexisting, matcherOut.length ) :
+					matcherOut
+			);
+			if ( postFinder ) {
+				postFinder( null, results, matcherOut, xml );
+			} else {
+				push.apply( results, matcherOut );
+			}
+		}
+	});
+}
+
+function matcherFromTokens( tokens ) {
+	var checkContext, matcher, j,
+		len = tokens.length,
+		leadingRelative = Expr.relative[ tokens[0].type ],
+		implicitRelative = leadingRelative || Expr.relative[" "],
+		i = leadingRelative ? 1 : 0,
+
+		// The foundational matcher ensures that elements are reachable from top-level context(s)
+		matchContext = addCombinator( function( elem ) {
+			return elem === checkContext;
+		}, implicitRelative, true ),
+		matchAnyContext = addCombinator( function( elem ) {
+			return indexOf( checkContext, elem ) > -1;
+		}, implicitRelative, true ),
+		matchers = [ function( elem, context, xml ) {
+			var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
+				(checkContext = context).nodeType ?
+					matchContext( elem, context, xml ) :
+					matchAnyContext( elem, context, xml ) );
+			// Avoid hanging onto element (issue #299)
+			checkContext = null;
+			return ret;
+		} ];
+
+	for ( ; i < len; i++ ) {
+		if ( (matcher = Expr.relative[ tokens[i].type ]) ) {
+			matchers = [ addCombinator(elementMatcher( matchers ), matcher) ];
+		} else {
+			matcher = Expr.filter[ tokens[i].type ].apply( null, tokens[i].matches );
+
+			// Return special upon seeing a positional matcher
+			if ( matcher[ expando ] ) {
+				// Find the next relative operator (if any) for proper handling
+				j = ++i;
+				for ( ; j < len; j++ ) {
+					if ( Expr.relative[ tokens[j].type ] ) {
+						break;
+					}
+				}
+				return setMatcher(
+					i > 1 && elementMatcher( matchers ),
+					i > 1 && toSelector(
+						// If the preceding token was a descendant combinator, insert an implicit any-element `*`
+						tokens.slice( 0, i - 1 ).concat({ value: tokens[ i - 2 ].type === " " ? "*" : "" })
+					).replace( rtrim, "$1" ),
+					matcher,
+					i < j && matcherFromTokens( tokens.slice( i, j ) ),
+					j < len && matcherFromTokens( (tokens = tokens.slice( j )) ),
+					j < len && toSelector( tokens )
+				);
+			}
+			matchers.push( matcher );
+		}
+	}
+
+	return elementMatcher( matchers );
+}
+
+function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
+	var bySet = setMatchers.length > 0,
+		byElement = elementMatchers.length > 0,
+		superMatcher = function( seed, context, xml, results, outermost ) {
+			var elem, j, matcher,
+				matchedCount = 0,
+				i = "0",
+				unmatched = seed && [],
+				setMatched = [],
+				contextBackup = outermostContext,
+				// We must always have either seed elements or outermost context
+				elems = seed || byElement && Expr.find["TAG"]( "*", outermost ),
+				// Use integer dirruns iff this is the outermost matcher
+				dirrunsUnique = (dirruns += contextBackup == null ? 1 : Math.random() || 0.1),
+				len = elems.length;
+
+			if ( outermost ) {
+				outermostContext = context !== document && context;
+			}
+
+			// Add elements passing elementMatchers directly to results
+			// Keep `i` a string if there are no elements so `matchedCount` will be "00" below
+			// Support: IE<9, Safari
+			// Tolerate NodeList properties (IE: "length"; Safari: <number>) matching elements by id
+			for ( ; i !== len && (elem = elems[i]) != null; i++ ) {
+				if ( byElement && elem ) {
+					j = 0;
+					while ( (matcher = elementMatchers[j++]) ) {
+						if ( matcher( elem, context, xml ) ) {
+							results.push( elem );
+							break;
+						}
+					}
+					if ( outermost ) {
+						dirruns = dirrunsUnique;
+					}
+				}
+
+				// Track unmatched elements for set filters
+				if ( bySet ) {
+					// They will have gone through all possible matchers
+					if ( (elem = !matcher && elem) ) {
+						matchedCount--;
+					}
+
+					// Lengthen the array for every element, matched or not
+					if ( seed ) {
+						unmatched.push( elem );
+					}
+				}
+			}
+
+			// Apply set filters to unmatched elements
+			matchedCount += i;
+			if ( bySet && i !== matchedCount ) {
+				j = 0;
+				while ( (matcher = setMatchers[j++]) ) {
+					matcher( unmatched, setMatched, context, xml );
+				}
+
+				if ( seed ) {
+					// Reintegrate element matches to eliminate the need for sorting
+					if ( matchedCount > 0 ) {
+						while ( i-- ) {
+							if ( !(unmatched[i] || setMatched[i]) ) {
+								setMatched[i] = pop.call( results );
+							}
+						}
+					}
+
+					// Discard index placeholder values to get only actual matches
+					setMatched = condense( setMatched );
+				}
+
+				// Add matches to results
+				push.apply( results, setMatched );
+
+				// Seedless set matches succeeding multiple successful matchers stipulate sorting
+				if ( outermost && !seed && setMatched.length > 0 &&
+					( matchedCount + setMatchers.length ) > 1 ) {
+
+					Sizzle.uniqueSort( results );
+				}
+			}
+
+			// Override manipulation of globals by nested matchers
+			if ( outermost ) {
+				dirruns = dirrunsUnique;
+				outermostContext = contextBackup;
+			}
+
+			return unmatched;
+		};
+
+	return bySet ?
+		markFunction( superMatcher ) :
+		superMatcher;
+}
+
+compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
+	var i,
+		setMatchers = [],
+		elementMatchers = [],
+		cached = compilerCache[ selector + " " ];
+
+	if ( !cached ) {
+		// Generate a function of recursive functions that can be used to check each element
+		if ( !match ) {
+			match = tokenize( selector );
+		}
+		i = match.length;
+		while ( i-- ) {
+			cached = matcherFromTokens( match[i] );
+			if ( cached[ expando ] ) {
+				setMatchers.push( cached );
+			} else {
+				elementMatchers.push( cached );
+			}
+		}
+
+		// Cache the compiled function
+		cached = compilerCache( selector, matcherFromGroupMatchers( elementMatchers, setMatchers ) );
+
+		// Save selector and tokenization
+		cached.selector = selector;
+	}
+	return cached;
+};
+
+/**
+ * A low-level selection function that works with Sizzle's compiled
+ *  selector functions
+ * @param {String|Function} selector A selector or a pre-compiled
+ *  selector function built with Sizzle.compile
+ * @param {Element} context
+ * @param {Array} [results]
+ * @param {Array} [seed] A set of elements to match against
+ */
+select = Sizzle.select = function( selector, context, results, seed ) {
+	var i, tokens, token, type, find,
+		compiled = typeof selector === "function" && selector,
+		match = !seed && tokenize( (selector = compiled.selector || selector) );
+
+	results = results || [];
+
+	// Try to minimize operations if there is no seed and only one group
+	if ( match.length === 1 ) {
+
+		// Take a shortcut and set the context if the root selector is an ID
+		tokens = match[0] = match[0].slice( 0 );
+		if ( tokens.length > 2 && (token = tokens[0]).type === "ID" &&
+				support.getById && context.nodeType === 9 && documentIsHTML &&
+				Expr.relative[ tokens[1].type ] ) {
+
+			context = ( Expr.find["ID"]( token.matches[0].replace(runescape, funescape), context ) || [] )[0];
+			if ( !context ) {
+				return results;
+
+			// Precompiled matchers will still verify ancestry, so step up a level
+			} else if ( compiled ) {
+				context = context.parentNode;
+			}
+
+			selector = selector.slice( tokens.shift().value.length );
+		}
+
+		// Fetch a seed set for right-to-left matching
+		i = matchExpr["needsContext"].test( selector ) ? 0 : tokens.length;
+		while ( i-- ) {
+			token = tokens[i];
+
+			// Abort if we hit a combinator
+			if ( Expr.relative[ (type = token.type) ] ) {
+				break;
+			}
+			if ( (find = Expr.find[ type ]) ) {
+				// Search, expanding context for leading sibling combinators
+				if ( (seed = find(
+					token.matches[0].replace( runescape, funescape ),
+					rsibling.test( tokens[0].type ) && testContext( context.parentNode ) || context
+				)) ) {
+
+					// If seed is empty or no tokens remain, we can return early
+					tokens.splice( i, 1 );
+					selector = seed.length && toSelector( tokens );
+					if ( !selector ) {
+						push.apply( results, seed );
+						return results;
+					}
+
+					break;
+				}
+			}
+		}
+	}
+
+	// Compile and execute a filtering function if one is not provided
+	// Provide `match` to avoid retokenization if we modified the selector above
+	( compiled || compile( selector, match ) )(
+		seed,
+		context,
+		!documentIsHTML,
+		results,
+		rsibling.test( selector ) && testContext( context.parentNode ) || context
+	);
+	return results;
+};
+
+// One-time assignments
+
+// Sort stability
+support.sortStable = expando.split("").sort( sortOrder ).join("") === expando;
+
+// Support: Chrome 14-35+
+// Always assume duplicates if they aren't passed to the comparison function
+support.detectDuplicates = !!hasDuplicate;
+
+// Initialize against the default document
+setDocument();
+
+// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27)
+// Detached nodes confoundingly follow *each other*
+support.sortDetached = assert(function( div1 ) {
+	// Should return 1, but returns 4 (following)
+	return div1.compareDocumentPosition( document.createElement("div") ) & 1;
+});
+
+// Support: IE<8
+// Prevent attribute/property "interpolation"
+// http://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
+if ( !assert(function( div ) {
+	div.innerHTML = "<a href='#'></a>";
+	return div.firstChild.getAttribute("href") === "#" ;
+}) ) {
+	addHandle( "type|href|height|width", function( elem, name, isXML ) {
+		if ( !isXML ) {
+			return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
+		}
+	});
+}
+
+// Support: IE<9
+// Use defaultValue in place of getAttribute("value")
+if ( !support.attributes || !assert(function( div ) {
+	div.innerHTML = "<input/>";
+	div.firstChild.setAttribute( "value", "" );
+	return div.firstChild.getAttribute( "value" ) === "";
+}) ) {
+	addHandle( "value", function( elem, name, isXML ) {
+		if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
+			return elem.defaultValue;
+		}
+	});
+}
+
+// Support: IE<9
+// Use getAttributeNode to fetch booleans when getAttribute lies
+if ( !assert(function( div ) {
+	return div.getAttribute("disabled") == null;
+}) ) {
+	addHandle( booleans, function( elem, name, isXML ) {
+		var val;
+		if ( !isXML ) {
+			return elem[ name ] === true ? name.toLowerCase() :
+					(val = elem.getAttributeNode( name )) && val.specified ?
+					val.value :
+				null;
+		}
+	});
+}
+
+return Sizzle;
+
+})( window );
+
+
+
+jQuery.find = Sizzle;
+jQuery.expr = Sizzle.selectors;
+jQuery.expr[":"] = jQuery.expr.pseudos;
+jQuery.unique = Sizzle.uniqueSort;
+jQuery.text = Sizzle.getText;
+jQuery.isXMLDoc = Sizzle.isXML;
+jQuery.contains = Sizzle.contains;
+
+
+
+var rneedsContext = jQuery.expr.match.needsContext;
+
+var rsingleTag = (/^<(\w+)\s*\/?>(?:<\/\1>|)$/);
+
+
+
+var risSimple = /^.[^:#\[\.,]*$/;
+
+// Implement the identical functionality for filter and not
+function winnow( elements, qualifier, not ) {
+	if ( jQuery.isFunction( qualifier ) ) {
+		return jQuery.grep( elements, function( elem, i ) {
+			/* jshint -W018 */
+			return !!qualifier.call( elem, i, elem ) !== not;
+		});
+
+	}
+
+	if ( qualifier.nodeType ) {
+		return jQuery.grep( elements, function( elem ) {
+			return ( elem === qualifier ) !== not;
+		});
+
+	}
+
+	if ( typeof qualifier === "string" ) {
+		if ( risSimple.test( qualifier ) ) {
+			return jQuery.filter( qualifier, elements, not );
+		}
+
+		qualifier = jQuery.filter( qualifier, elements );
+	}
+
+	return jQuery.grep( elements, function( elem ) {
+		return ( indexOf.call( qualifier, elem ) >= 0 ) !== not;
+	});
+}
+
+jQuery.filter = function( expr, elems, not ) {
+	var elem = elems[ 0 ];
+
+	if ( not ) {
+		expr = ":not(" + expr + ")";
+	}
+
+	return elems.length === 1 && elem.nodeType === 1 ?
+		jQuery.find.matchesSelector( elem, expr ) ? [ elem ] : [] :
+		jQuery.find.matches( expr, jQuery.grep( elems, function( elem ) {
+			return elem.nodeType === 1;
+		}));
+};
+
+jQuery.fn.extend({
+	find: function( selector ) {
+		var i,
+			len = this.length,
+			ret = [],
+			self = this;
+
+		if ( typeof selector !== "string" ) {
+			return this.pushStack( jQuery( selector ).filter(function() {
+				for ( i = 0; i < len; i++ ) {
+					if ( jQuery.contains( self[ i ], this ) ) {
+						return true;
+					}
+				}
+			}) );
+		}
+
+		for ( i = 0; i < len; i++ ) {
+			jQuery.find( selector, self[ i ], ret );
+		}
+
+		// Needed because $( selector, context ) becomes $( context ).find( selector )
+		ret = this.pushStack( len > 1 ? jQuery.unique( ret ) : ret );
+		ret.selector = this.selector ? this.selector + " " + selector : selector;
+		return ret;
+	},
+	filter: function( selector ) {
+		return this.pushStack( winnow(this, selector || [], false) );
+	},
+	not: function( selector ) {
+		return this.pushStack( winnow(this, selector || [], true) );
+	},
+	is: function( selector ) {
+		return !!winnow(
+			this,
+
+			// If this is a positional/relative selector, check membership in the returned set
+			// so $("p:first").is("p:last") won't return true for a doc with two "p".
+			typeof selector === "string" && rneedsContext.test( selector ) ?
+				jQuery( selector ) :
+				selector || [],
+			false
+		).length;
+	}
+});
+
+
+// Initialize a jQuery object
+
+
+// A central reference to the root jQuery(document)
+var rootjQuery,
+
+	// A simple way to check for HTML strings
+	// Prioritize #id over <tag> to avoid XSS via location.hash (#9521)
+	// Strict HTML recognition (#11290: must start with <)
+	rquickExpr = /^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,
+
+	init = jQuery.fn.init = function( selector, context ) {
+		var match, elem;
+
+		// HANDLE: $(""), $(null), $(undefined), $(false)
+		if ( !selector ) {
+			return this;
+		}
+
+		// Handle HTML strings
+		if ( typeof selector === "string" ) {
+			if ( selector[0] === "<" && selector[ selector.length - 1 ] === ">" && selector.length >= 3 ) {
+				// Assume that strings that start and end with <> are HTML and skip the regex check
+				match = [ null, selector, null ];
+
+			} else {
+				match = rquickExpr.exec( selector );
+			}
+
+			// Match html or make sure no context is specified for #id
+			if ( match && (match[1] || !context) ) {
+
+				// HANDLE: $(html) -> $(array)
+				if ( match[1] ) {
+					context = context instanceof jQuery ? context[0] : context;
+
+					// Option to run scripts is true for back-compat
+					// Intentionally let the error be thrown if parseHTML is not present
+					jQuery.merge( this, jQuery.parseHTML(
+						match[1],
+						context && context.nodeType ? context.ownerDocument || context : document,
+						true
+					) );
+
+					// HANDLE: $(html, props)
+					if ( rsingleTag.test( match[1] ) && jQuery.isPlainObject( context ) ) {
+						for ( match in context ) {
+							// Properties of context are called as methods if possible
+							if ( jQuery.isFunction( this[ match ] ) ) {
+								this[ match ]( context[ match ] );
+
+							// ...and otherwise set as attributes
+							} else {
+								this.attr( match, context[ match ] );
+							}
+						}
+					}
+
+					return this;
+
+				// HANDLE: $(#id)
+				} else {
+					elem = document.getElementById( match[2] );
+
+					// Support: Blackberry 4.6
+					// gEBID returns nodes no longer in the document (#6963)
+					if ( elem && elem.parentNode ) {
+						// Inject the element directly into the jQuery object
+						this.length = 1;
+						this[0] = elem;
+					}
+
+					this.context = document;
+					this.selector = selector;
+					return this;
+				}
+
+			// HANDLE: $(expr, $(...))
+			} else if ( !context || context.jquery ) {
+				return ( context || rootjQuery ).find( selector );
+
+			// HANDLE: $(expr, context)
+			// (which is just equivalent to: $(context).find(expr)
+			} else {
+				return this.constructor( context ).find( selector );
+			}
+
+		// HANDLE: $(DOMElement)
+		} else if ( selector.nodeType ) {
+			this.context = this[0] = selector;
+			this.length = 1;
+			return this;
+
+		// HANDLE: $(function)
+		// Shortcut for document ready
+		} else if ( jQuery.isFunction( selector ) ) {
+			return typeof rootjQuery.ready !== "undefined" ?
+				rootjQuery.ready( selector ) :
+				// Execute immediately if ready is not present
+				selector( jQuery );
+		}
+
+		if ( selector.selector !== undefined ) {
+			this.selector = selector.selector;
+			this.context = selector.context;
+		}
+
+		return jQuery.makeArray( selector, this );
+	};
+
+// Give the init function the jQuery prototype for later instantiation
+init.prototype = jQuery.fn;
+
+// Initialize central reference
+rootjQuery = jQuery( document );
+
+
+var rparentsprev = /^(?:parents|prev(?:Until|All))/,
+	// Methods guaranteed to produce a unique set when starting from a unique set
+	guaranteedUnique = {
+		children: true,
+		contents: true,
+		next: true,
+		prev: true
+	};
+
+jQuery.extend({
+	dir: function( elem, dir, until ) {
+		var matched = [],
+			truncate = until !== undefined;
+
+		while ( (elem = elem[ dir ]) && elem.nodeType !== 9 ) {
+			if ( elem.nodeType === 1 ) {
+				if ( truncate && jQuery( elem ).is( until ) ) {
+					break;
+				}
+				matched.push( elem );
+			}
+		}
+		return matched;
+	},
+
+	sibling: function( n, elem ) {
+		var matched = [];
+
+		for ( ; n; n = n.nextSibling ) {
+			if ( n.nodeType === 1 && n !== elem ) {
+				matched.push( n );
+			}
+		}
+
+		return matched;
+	}
+});
+
+jQuery.fn.extend({
+	has: function( target ) {
+		var targets = jQuery( target, this ),
+			l = targets.length;
+
+		return this.filter(function() {
+			var i = 0;
+			for ( ; i < l; i++ ) {
+				if ( jQuery.contains( this, targets[i] ) ) {
+					return true;
+				}
+			}
+		});
+	},
+
+	closest: function( selectors, context ) {
+		var cur,
+			i = 0,
+			l = this.length,
+			matched = [],
+			pos = rneedsContext.test( selectors ) || typeof selectors !== "string" ?
+				jQuery( selectors, context || this.context ) :
+				0;
+
+		for ( ; i < l; i++ ) {
+			for ( cur = this[i]; cur && cur !== context; cur = cur.parentNode ) {
+				// Always skip document fragments
+				if ( cur.nodeType < 11 && (pos ?
+					pos.index(cur) > -1 :
+
+					// Don't pass non-elements to Sizzle
+					cur.nodeType === 1 &&
+						jQuery.find.matchesSelector(cur, selectors)) ) {
+
+					matched.push( cur );
+					break;
+				}
+			}
+		}
+
+		return this.pushStack( matched.length > 1 ? jQuery.unique( matched ) : matched );
+	},
+
+	// Determine the position of an element within the set
+	index: function( elem ) {
+
+		// No argument, return index in parent
+		if ( !elem ) {
+			return ( this[ 0 ] && this[ 0 ].parentNode ) ? this.first().prevAll().length : -1;
+		}
+
+		// Index in selector
+		if ( typeof elem === "string" ) {
+			return indexOf.call( jQuery( elem ), this[ 0 ] );
+		}
+
+		// Locate the position of the desired element
+		return indexOf.call( this,
+
+			// If it receives a jQuery object, the first element is used
+			elem.jquery ? elem[ 0 ] : elem
+		);
+	},
+
+	add: function( selector, context ) {
+		return this.pushStack(
+			jQuery.unique(
+				jQuery.merge( this.get(), jQuery( selector, context ) )
+			)
+		);
+	},
+
+	addBack: function( selector ) {
+		return this.add( selector == null ?
+			this.prevObject : this.prevObject.filter(selector)
+		);
+	}
+});
+
+function sibling( cur, dir ) {
+	while ( (cur = cur[dir]) && cur.nodeType !== 1 ) {}
+	return cur;
+}
+
+jQuery.each({
+	parent: function( elem ) {
+		var parent = elem.parentNode;
+		return parent && parent.nodeType !== 11 ? parent : null;
+	},
+	parents: function( elem ) {
+		return jQuery.dir( elem, "parentNode" );
+	},
+	parentsUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "parentNode", until );
+	},
+	next: function( elem ) {
+		return sibling( elem, "nextSibling" );
+	},
+	prev: function( elem ) {
+		return sibling( elem, "previousSibling" );
+	},
+	nextAll: function( elem ) {
+		return jQuery.dir( elem, "nextSibling" );
+	},
+	prevAll: function( elem ) {
+		return jQuery.dir( elem, "previousSibling" );
+	},
+	nextUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "nextSibling", until );
+	},
+	prevUntil: function( elem, i, until ) {
+		return jQuery.dir( elem, "previousSibling", until );
+	},
+	siblings: function( elem ) {
+		return jQuery.sibling( ( elem.parentNode || {} ).firstChild, elem );
+	},
+	children: function( elem ) {
+		return jQuery.sibling( elem.firstChild );
+	},
+	contents: function( elem ) {
+		return elem.contentDocument || jQuery.merge( [], elem.childNodes );
+	}
+}, function( name, fn ) {
+	jQuery.fn[ name ] = function( until, selector ) {
+		var matched = jQuery.map( this, fn, until );
+
+		if ( name.slice( -5 ) !== "Until" ) {
+			selector = until;
+		}
+
+		if ( selector && typeof selector === "string" ) {
+			matched = jQuery.filter( selector, matched );
+		}
+
+		if ( this.length > 1 ) {
+			// Remove duplicates
+			if ( !guaranteedUnique[ name ] ) {
+				jQuery.unique( matched );
+			}
+
+			// Reverse order for parents* and prev-derivatives
+			if ( rparentsprev.test( name ) ) {
+				matched.reverse();
+			}
+		}
+
+		return this.pushStack( matched );
+	};
+});
+var rnotwhite = (/\S+/g);
+
+
+
+// String to Object options format cache
+var optionsCache = {};
+
+// Convert String-formatted options into Object-formatted ones and store in cache
+function createOptions( options ) {
+	var object = optionsCache[ options ] = {};
+	jQuery.each( options.match( rnotwhite ) || [], function( _, flag ) {
+		object[ flag ] = true;
+	});
+	return object;
+}
+
+/*
+ * Create a callback list using the following parameters:
+ *
+ *	options: an optional list of space-separated options that will change how
+ *			the callback list behaves or a more traditional option object
+ *
+ * By default a callback list will act like an event callback list and can be
+ * "fired" multiple times.
+ *
+ * Possible options:
+ *
+ *	once:			will ensure the callback list can only be fired once (like a Deferred)
+ *
+ *	memory:			will keep track of previous values and will call any callback added
+ *					after the list has been fired right away with the latest "memorized"
+ *					values (like a Deferred)
+ *
+ *	unique:			will ensure a callback can only be added once (no duplicate in the list)
+ *
+ *	stopOnFalse:	interrupt callings when a callback returns false
+ *
+ */
+jQuery.Callbacks = function( options ) {
+
+	// Convert options from String-formatted to Object-formatted if needed
+	// (we check in cache first)
+	options = typeof options === "string" ?
+		( optionsCache[ options ] || createOptions( options ) ) :
+		jQuery.extend( {}, options );
+
+	var // Last fire value (for non-forgettable lists)
+		memory,
+		// Flag to know if list was already fired
+		fired,
+		// Flag to know if list is currently firing
+		firing,
+		// First callback to fire (used internally by add and fireWith)
+		firingStart,
+		// End of the loop when firing
+		firingLength,
+		// Index of currently firing callback (modified by remove if needed)
+		firingIndex,
+		// Actual callback list
+		list = [],
+		// Stack of fire calls for repeatable lists
+		stack = !options.once && [],
+		// Fire callbacks
+		fire = function( data ) {
+			memory = options.memory && data;
+			fired = true;
+			firingIndex = firingStart || 0;
+			firingStart = 0;
+			firingLength = list.length;
+			firing = true;
+			for ( ; list && firingIndex < firingLength; firingIndex++ ) {
+				if ( list[ firingIndex ].apply( data[ 0 ], data[ 1 ] ) === false && options.stopOnFalse ) {
+					memory = false; // To prevent further calls using add
+					break;
+				}
+			}
+			firing = false;
+			if ( list ) {
+				if ( stack ) {
+					if ( stack.length ) {
+						fire( stack.shift() );
+					}
+				} else if ( memory ) {
+					list = [];
+				} else {
+					self.disable();
+				}
+			}
+		},
+		// Actual Callbacks object
+		self = {
+			// Add a callback or a collection of callbacks to the list
+			add: function() {
+				if ( list ) {
+					// First, we save the current length
+					var start = list.length;
+					(function add( args ) {
+						jQuery.each( args, function( _, arg ) {
+							var type = jQuery.type( arg );
+							if ( type === "function" ) {
+								if ( !options.unique || !self.has( arg ) ) {
+									list.push( arg );
+								}
+							} else if ( arg && arg.length && type !== "string" ) {
+								// Inspect recursively
+								add( arg );
+							}
+						});
+					})( arguments );
+					// Do we need to add the callbacks to the
+					// current firing batch?
+					if ( firing ) {
+						firingLength = list.length;
+					// With

<TRUNCATED>

[19/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/js/index.js.orig
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/index.js.orig b/flink-runtime-web/src/main/resources/web/js/index.js.orig
new file mode 100644
index 0000000..018e743
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/js/index.js.orig
@@ -0,0 +1,1150 @@
+angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
+  $rootScope.sidebarVisible = false;
+  return $rootScope.showSidebar = function() {
+    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
+    return $rootScope.sidebarClass = 'force-show';
+  };
+}]).value('flinkConfig', {
+  "refresh-interval": 10000
+}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
+  return MainService.loadConfig().then(function(config) {
+    angular.extend(flinkConfig, config);
+    JobsService.listJobs();
+    return $interval(function() {
+      return JobsService.listJobs();
+    }, flinkConfig["refresh-interval"]);
+  });
+}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
+  return $uiViewScrollProvider.useAnchorScroll();
+}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
+  $stateProvider.state("overview", {
+    url: "/overview",
+    views: {
+      main: {
+        templateUrl: "partials/overview.html",
+        controller: 'OverviewController'
+      }
+    }
+  }).state("running-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: true,
+    views: {
+      main: {
+        templateUrl: "partials/jobs/job.html",
+        controller: 'SingleJobController'
+      }
+    }
+  }).state("single-job.plan", {
+    url: "",
+    abstract: true,
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.plan.html",
+        controller: 'JobPlanController'
+      }
+    }
+  }).state("single-job.plan.overview", {
+    url: "",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
+        controller: 'JobPlanOverviewController'
+      }
+    }
+  }).state("single-job.plan.accumulators", {
+    url: "/accumulators",
+    views: {
+      'node-details': {
+        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
+        controller: 'JobPlanAccumulatorsController'
+      }
+    }
+  }).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.statistics", {
+    url: "/statistics",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.statistics.html"
+      }
+    }
+  }).state("single-job.exceptions", {
+    url: "/exceptions",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.exceptions.html",
+        controller: 'JobExceptionsController'
+      }
+    }
+  }).state("single-job.properties", {
+    url: "/properties",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.properties.html",
+        controller: 'JobPropertiesController'
+      }
+    }
+  }).state("single-job.config", {
+    url: "/config",
+    views: {
+      details: {
+        templateUrl: "partials/jobs/job.config.html"
+      }
+    }
+  }).state("taskmanagers", {
+    url: "/taskmanagers",
+    views: {
+      main: {
+        templateUrl: "partials/taskmanagers/index.html",
+        controller: 'TaskManagersController'
+      }
+    }
+  }).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"
+      }
+    }
+  }).state("jobmanager.logfile", {
+    url: "/logfile",
+    views: {
+      details: {
+        templateUrl: "partials/jobmanager/logfile.html"
+      }
+    }
+  });
+  return $urlRouterProvider.otherwise("/overview");
+}]);
+
+angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
+  return {
+    transclude: true,
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: "@"
+    },
+    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'label label-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
+  return {
+    replace: true,
+    scope: {
+      getLabelClass: "&",
+      status: '@'
+    },
+    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
+    link: function(scope, element, attrs) {
+      return scope.getLabelClass = function() {
+        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
+      };
+    }
+  };
+}]).directive('tableProperty', function() {
+  return {
+    replace: true,
+    scope: {
+      value: '='
+    },
+    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
+  };
+});
+
+angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
+  var amDurationFormatExtendedFilter;
+  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
+    if (typeof value === "undefined" || value === null) {
+      return "";
+    }
+    return moment.duration(value, format).format(durationFormat, {
+      trim: false
+    });
+  };
+  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
+  return amDurationFormatExtendedFilter;
+}]).filter("humanizeText", function() {
+  return function(text) {
+    if (text) {
+      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
+    } else {
+      return '';
+    }
+  };
+}).filter("bytes", function() {
+  return function(bytes, precision) {
+    var number, units;
+    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
+      return "-";
+    }
+    if (typeof precision === "undefined") {
+      precision = 1;
+    }
+    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
+    number = Math.floor(Math.log(bytes) / Math.log(1024));
+    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
+  };
+});
+
+angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("/config").success(function(data, status, headers, config) {
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
+  return JobManagerConfigService.loadConfig().then(function(data) {
+    if ($scope.jobmanager == null) {
+      $scope.jobmanager = {};
+    }
+    return $scope.jobmanager['config'] = data;
+  });
+}]);
+
+angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var config;
+  config = {};
+  this.loadConfig = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("/jobmanager/config").success(function(data, status, headers, config) {
+      config = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('running');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  $scope.jobObserver = function() {
+    return $scope.jobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  return $scope.jobObserver();
+}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
+  var refresher;
+  console.log('SingleJobController');
+  $scope.jobid = $stateParams.jobid;
+  $scope.job = null;
+  $scope.plan = null;
+  $scope.vertices = null;
+  JobsService.loadJob($stateParams.jobid).then(function(data) {
+    $scope.job = data;
+    $scope.plan = data.plan;
+    return $scope.vertices = data.vertices;
+  });
+  refresher = $interval(function() {
+    return JobsService.loadJob($stateParams.jobid).then(function(data) {
+      $scope.job = data;
+      return $scope.$broadcast('reload');
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    $scope.job = null;
+    $scope.plan = null;
+    $scope.vertices = null;
+    return $interval.cancel(refresher);
+  });
+}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobPlanController');
+  $scope.nodeid = null;
+  $scope.nodeUnfolded = false;
+  $scope.stateList = JobsService.stateList();
+  $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      $scope.accumulators = null;
+      return $scope.$broadcast('reload');
+    } else {
+      $scope.nodeid = null;
+      $scope.nodeUnfolded = false;
+      $scope.vertex = null;
+      $scope.subtasks = null;
+      return $scope.accumulators = null;
+    }
+  };
+  $scope.deactivateNode = function() {
+    $scope.nodeid = null;
+    $scope.nodeUnfolded = false;
+    $scope.vertex = null;
+    $scope.subtasks = null;
+    return $scope.accumulators = null;
+  };
+  return $scope.toggleFold = function() {
+    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
+  };
+}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanOverviewController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
+    JobsService.getSubtasks($scope.nodeid).then(function(data) {
+      return $scope.subtasks = data;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanOverviewController');
+    if ($scope.nodeid) {
+      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
+        return $scope.subtasks = data;
+      });
+    }
+  });
+}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPlanAccumulatorsController');
+  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
+    JobsService.getAccumulators($scope.nodeid).then(function(data) {
+      $scope.accumulators = data.main;
+      return $scope.subtaskAccumulators = data.subtasks;
+    });
+  }
+  return $scope.$on('reload', function(event) {
+    console.log('JobPlanAccumulatorsController');
+    if ($scope.nodeid) {
+      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
+        $scope.accumulators = data.main;
+        return $scope.subtaskAccumulators = data.subtasks;
+      });
+    }
+  });
+}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  console.log('JobTimelineVertexController');
+  JobsService.getVertex($stateParams.vertexId).then(function(data) {
+    return $scope.vertex = data;
+  });
+  return $scope.$on('reload', function(event) {
+    console.log('JobTimelineVertexController');
+    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
+      return $scope.vertex = data;
+    });
+  });
+}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
+  return JobsService.loadExceptions().then(function(data) {
+    return $scope.exceptions = data;
+  });
+}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
+  console.log('JobPropertiesController');
+  return $scope.changeNode = function(nodeid) {
+    if (nodeid !== $scope.nodeid) {
+      $scope.nodeid = nodeid;
+      return JobsService.getNode(nodeid).then(function(data) {
+        return $scope.node = data;
+      });
+    } else {
+      $scope.nodeid = null;
+      return $scope.node = null;
+    }
+  };
+}]);
+
+angular.module('flinkApp').directive('vertex', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
+    scope: {
+      data: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data.subtasks, function(subtask, i) {
+          var times;
+          times = [
+            {
+              label: "Scheduled",
+              color: "#666",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["SCHEDULED"],
+              ending_time: subtask.timestamps["DEPLOYING"],
+              type: 'regular'
+            }, {
+              label: "Deploying",
+              color: "#aaa",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["DEPLOYING"],
+              ending_time: subtask.timestamps["RUNNING"],
+              type: 'regular'
+            }
+          ];
+          if (subtask.timestamps["FINISHED"] > 0) {
+            times.push({
+              label: "Running",
+              color: "#ddd",
+              borderColor: "#555",
+              starting_time: subtask.timestamps["RUNNING"],
+              ending_time: subtask.timestamps["FINISHED"],
+              type: 'regular'
+            });
+          }
+          return testData.push({
+            label: "(" + subtask.subtask + ") " + subtask.host,
+            times: times
+          });
+        });
+        chart = d3.timeline().stack().tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("single").labelFormat(function(label) {
+          return label;
+        }).margin({
+          left: 100,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).relativeTime();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      analyzeTime(scope.data);
+    }
+  };
+}]).directive('timeline', ["$state", function($state) {
+  return {
+    template: "<svg class='timeline' width='0' height='0'></svg>",
+    scope: {
+      vertices: "=",
+      jobid: "="
+    },
+    link: function(scope, elem, attrs) {
+      var analyzeTime, containerW, svgEl, translateLabel;
+      svgEl = elem.children()[0];
+      containerW = elem.width();
+      angular.element(svgEl).attr('width', containerW);
+      translateLabel = function(label) {
+        return label.replace("&gt;", ">");
+      };
+      analyzeTime = function(data) {
+        var chart, svg, testData;
+        d3.select(svgEl).selectAll("*").remove();
+        testData = [];
+        angular.forEach(data, function(vertex) {
+          if (vertex['start-time'] > -1) {
+            if (vertex.type === 'scheduled') {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#cccccc",
+                    borderColor: "#555555",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    type: vertex.type
+                  }
+                ]
+              });
+            } else {
+              return testData.push({
+                times: [
+                  {
+                    label: translateLabel(vertex.name),
+                    color: "#d9f1f7",
+                    borderColor: "#62cdea",
+                    starting_time: vertex['start-time'],
+                    ending_time: vertex['end-time'],
+                    link: vertex.id,
+                    type: vertex.type
+                  }
+                ]
+              });
+            }
+          }
+        });
+        chart = d3.timeline().stack().click(function(d, i, datum) {
+          if (d.link) {
+            return $state.go("single-job.timeline.vertex", {
+              jobid: scope.jobid,
+              vertexId: d.link
+            });
+          }
+        }).tickFormat({
+          format: d3.time.format("%L"),
+          tickSize: 1
+        }).prefix("main").margin({
+          left: 0,
+          right: 0,
+          top: 0,
+          bottom: 0
+        }).itemHeight(30).showBorderLine().showHourTimeline();
+        return svg = d3.select(svgEl).datum(testData).call(chart);
+      };
+      scope.$watch(attrs.vertices, function(data) {
+        if (data) {
+          return analyzeTime(data);
+        }
+      });
+    }
+  };
+}]).directive('jobPlan', ["$timeout", function($timeout) {
+  return {
+    template: "<svg class='graph' width='500' height='400'><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: '=',
+      setNode: '&'
+    },
+    link: function(scope, elem, attrs) {
+      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
+      g = null;
+      mainZoom = d3.behavior.zoom();
+      subgraphs = [];
+      jobid = attrs.jobid;
+      mainSvgElement = elem.children()[0];
+      mainG = elem.children().children()[0];
+      mainTmpElement = elem.children()[1];
+      d3mainSvg = d3.select(mainSvgElement);
+      d3mainSvgG = d3.select(mainG);
+      d3tmpSvg = d3.select(mainTmpElement);
+      containerW = elem.width();
+      angular.element(elem.children()[0]).width(containerW);
+      scope.zoomIn = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() < 2.99) {
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
+          mainZoom.scale(mainZoom.scale() + 0.1);
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      scope.zoomOut = function() {
+        var translate, v1, v2;
+        if (mainZoom.scale() > 0.31) {
+          mainZoom.scale(mainZoom.scale() - 0.1);
+          translate = mainZoom.translate();
+          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
+          mainZoom.translate([v1, v2]);
+          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
+        }
+      };
+      createLabelEdge = function(el) {
+        var labelValue;
+        labelValue = "";
+        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
+          labelValue += "<div class='edge-label'>";
+          if (el.ship_strategy != null) {
+            labelValue += el.ship_strategy;
+          }
+          if (el.temp_mode !== undefined) {
+            labelValue += " (" + el.temp_mode + ")";
+          }
+          if (el.local_strategy !== undefined) {
+            labelValue += ",<br>" + el.local_strategy;
+          }
+          labelValue += "</div>";
+        }
+        return labelValue;
+      };
+      isSpecialIterationNode = function(info) {
+        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
+      };
+      getNodeType = function(el, info) {
+        if (info === "mirror") {
+          return 'node-mirror';
+        } else if (isSpecialIterationNode(info)) {
+          return 'node-iteration';
+        } else {
+          return 'node-normal';
+        }
+      };
+      createLabelNode = function(el, info, maxW, maxH) {
+        var labelValue, stepName;
+        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
+        if (info === "mirror") {
+          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
+        } else {
+          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
+        }
+        if (el.description === "") {
+          labelValue += "";
+        } else {
+          stepName = el.description;
+          stepName = shortenString(stepName);
+          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
+        }
+        if (el.step_function != null) {
+          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
+        } else {
+          if (isSpecialIterationNode(info)) {
+            labelValue += "<h5>" + info + " Node</h5>";
+          }
+          if (el.parallelism !== "") {
+            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
+          }
+          if (el.operator !== undefined) {
+            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
+          }
+        }
+        labelValue += "</div>";
+        return labelValue;
+      };
+      extendLabelNodeForIteration = function(id, maxW, maxH) {
+        var labelValue, svgID;
+        svgID = "svg-" + id;
+        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
+        return labelValue;
+      };
+      shortenString = function(s) {
+        var sbr;
+        if (s.charAt(0) === "<") {
+          s = s.replace("<", "&lt;");
+          s = s.replace(">", "&gt;");
+        }
+        sbr = "";
+        while (s.length > 30) {
+          sbr = sbr + s.substring(0, 30) + "<br>";
+          s = s.substring(30, s.length);
+        }
+        sbr = sbr + s;
+        return sbr;
+      };
+      createNode = function(g, data, el, isParent, maxW, maxH) {
+        if (isParent == null) {
+          isParent = false;
+        }
+        if (el.id === data.partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "partialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "partialSolution")
+          });
+        } else if (el.id === data.next_partial_solution) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextPartialSolution")
+          });
+        } else if (el.id === data.workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "workset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "workset")
+          });
+        } else if (el.id === data.next_workset) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "nextWorkset", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "nextWorkset")
+          });
+        } else if (el.id === data.solution_set) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionSet", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionSet")
+          });
+        } else if (el.id === data.solution_delta) {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "solutionDelta", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "solutionDelta")
+          });
+        } else {
+          return g.setNode(el.id, {
+            label: createLabelNode(el, "", maxW, maxH),
+            labelType: 'html',
+            "class": getNodeType(el, "")
+          });
+        }
+      };
+      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
+        var missingNode;
+        if (existingNodes.indexOf(pred.id) !== -1) {
+          return g.setEdge(pred.id, el.id, {
+            label: createLabelEdge(pred),
+            labelType: 'html',
+            arrowhead: 'normal'
+          });
+        } else {
+          missingNode = searchForNode(data, pred.id);
+          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
+            missingNodes.push(missingNode.id);
+            g.setNode(missingNode.id, {
+              label: createLabelNode(missingNode, "mirror"),
+              labelType: 'html',
+              "class": getNodeType(missingNode, 'mirror')
+            });
+            return g.setEdge(missingNode.id, el.id, {
+              label: createLabelEdge(missingNode),
+              labelType: 'html'
+            });
+          }
+        }
+      };
+      loadJsonToDagre = function(g, data) {
+        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
+        existingNodes = [];
+        missingNodes = [];
+        if (data.nodes != null) {
+          toIterate = data.nodes;
+        } else {
+          toIterate = data.step_function;
+          isParent = true;
+        }
+        for (k = 0, len = toIterate.length; k < len; k++) {
+          el = toIterate[k];
+          maxW = 0;
+          maxH = 0;
+          if (el.step_function) {
+            sg = new dagreD3.graphlib.Graph({
+              multigraph: true,
+              compound: true
+            }).setGraph({
+              nodesep: 20,
+              edgesep: 0,
+              ranksep: 20,
+              rankdir: "LR",
+              marginx: 10,
+              marginy: 10
+            });
+            subgraphs[el.id] = sg;
+            loadJsonToDagre(sg, el);
+            r = new dagreD3.render();
+            d3tmpSvg.select('g').call(r, sg);
+            maxW = sg.graph().width;
+            maxH = sg.graph().height;
+            angular.element(mainTmpElement).empty();
+          }
+          createNode(g, data, el, isParent, maxW, maxH);
+          existingNodes.push(el.id);
+          if (el.inputs != null) {
+            ref = el.inputs;
+            for (l = 0, len1 = ref.length; l < len1; l++) {
+              pred = ref[l];
+              createEdge(g, data, el, existingNodes, pred, missingNodes);
+            }
+          }
+        }
+        return g;
+      };
+      searchForNode = function(data, nodeID) {
+        var el, i, j;
+        for (i in data.nodes) {
+          el = data.nodes[i];
+          if (el.id === nodeID) {
+            return el;
+          }
+          if (el.step_function != null) {
+            for (j in el.step_function) {
+              if (el.step_function[j].id === nodeID) {
+                return el.step_function[j];
+              }
+            }
+          }
+        }
+      };
+      drawGraph = function(data) {
+        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
+        g = new dagreD3.graphlib.Graph({
+          multigraph: true,
+          compound: true
+        }).setGraph({
+          nodesep: 70,
+          edgesep: 0,
+          ranksep: 50,
+          rankdir: "LR",
+          marginx: 40,
+          marginy: 40
+        });
+        loadJsonToDagre(g, data);
+        renderer = new dagreD3.render();
+        d3mainSvgG.call(renderer, g);
+        for (i in subgraphs) {
+          sg = subgraphs[i];
+          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
+        }
+        newScale = 0.5;
+        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
+        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
+        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
+        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
+        mainZoom.on("zoom", function() {
+          var ev;
+          ev = d3.event;
+          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
+        });
+        mainZoom(d3mainSvg);
+        return d3mainSvgG.selectAll('.node').on('click', function(d) {
+          return scope.setNode({
+            nodeid: d
+          });
+        });
+      };
+      scope.$watch(attrs.plan, function(newPlan) {
+        if (newPlan) {
+          return drawGraph(newPlan);
+        }
+      });
+    }
+  };
+}]);
+
+angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
+  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
+  currentJob = null;
+  currentPlan = null;
+  deferreds = {};
+  jobs = {
+    running: [],
+    finished: [],
+    cancelled: [],
+    failed: []
+  };
+  jobObservers = [];
+  notifyObservers = function() {
+    return angular.forEach(jobObservers, function(callback) {
+      return callback();
+    });
+  };
+  this.registerObserver = function(callback) {
+    return jobObservers.push(callback);
+  };
+  this.unRegisterObserver = function(callback) {
+    var index;
+    index = jobObservers.indexOf(callback);
+    return jobObservers.splice(index, 1);
+  };
+  this.stateList = function() {
+    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
+  };
+  this.translateLabelState = function(state) {
+    switch (state.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(list) {
+    return angular.forEach(list, function(item, jobKey) {
+      if (!(item['end-time'] > -1)) {
+        return item['end-time'] = item['start-time'] + item['duration'];
+      }
+    });
+  };
+  this.processVertices = function(data) {
+    angular.forEach(data.vertices, function(vertex, i) {
+      return vertex.type = 'regular';
+    });
+    return data.vertices.unshift({
+      name: 'Scheduled',
+      'start-time': data.timestamps['CREATED'],
+      'end-time': data.timestamps['CREATED'] + 1,
+      type: 'scheduled'
+    });
+  };
+  this.listJobs = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("/joboverview").success((function(_this) {
+      return function(data, status, headers, config) {
+        angular.forEach(data, function(list, listKey) {
+          switch (listKey) {
+            case 'running':
+              return jobs.running = _this.setEndTimes(list);
+            case 'finished':
+              return jobs.finished = _this.setEndTimes(list);
+            case 'cancelled':
+              return jobs.cancelled = _this.setEndTimes(list);
+            case 'failed':
+              return jobs.failed = _this.setEndTimes(list);
+          }
+        });
+        deferred.resolve(jobs);
+        return notifyObservers();
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getJobs = function(type) {
+    return jobs[type];
+  };
+  this.getAllJobs = function() {
+    return jobs;
+  };
+  this.loadJob = function(jobid) {
+    currentJob = null;
+    deferreds.job = $q.defer();
+    $http.get("/jobs/" + jobid).success((function(_this) {
+      return function(data, status, headers, config) {
+        _this.setEndTimes(data.vertices);
+        _this.processVertices(data);
+        return $http.get("/jobs/" + jobid + "/config").success(function(jobConfig) {
+          data = angular.extend(data, jobConfig);
+          currentJob = data;
+          return deferreds.job.resolve(currentJob);
+        });
+      };
+    })(this));
+    return deferreds.job.promise;
+  };
+  this.getNode = function(nodeid) {
+    var deferred, seekNode;
+    seekNode = function(nodeid, data) {
+      var j, len, node, sub;
+      for (j = 0, len = data.length; j < len; j++) {
+        node = data[j];
+        if (node.id === nodeid) {
+          return node;
+        }
+        if (node.step_function) {
+          sub = seekNode(nodeid, node.step_function);
+        }
+        if (sub) {
+          return sub;
+        }
+      }
+      return null;
+    };
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var foundNode;
+        foundNode = seekNode(nodeid, currentJob.plan.nodes);
+        foundNode.vertex = _this.seekVertex(nodeid);
+        return deferred.resolve(foundNode);
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.seekVertex = function(nodeid) {
+    var j, len, ref, vertex;
+    ref = currentJob.vertices;
+    for (j = 0, len = ref.length; j < len; j++) {
+      vertex = ref[j];
+      if (vertex.id === nodeid) {
+        return vertex;
+      }
+    }
+    return null;
+  };
+  this.getVertex = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        var vertex;
+        vertex = _this.seekVertex(vertexid);
+        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
+          vertex.subtasks = data.subtasks;
+          return deferred.resolve(vertex);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getSubtasks = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
+          var subtasks;
+          subtasks = data.subtasks;
+          return deferred.resolve(subtasks);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.getAccumulators = function(vertexid) {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
+          var accumulators;
+          accumulators = data['user-accumulators'];
+          return $http.get("/jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
+            var subtaskAccumulators;
+            subtaskAccumulators = data.subtasks;
+            return deferred.resolve({
+              main: accumulators,
+              subtasks: subtaskAccumulators
+            });
+          });
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  this.loadExceptions = function() {
+    var deferred;
+    deferred = $q.defer();
+    deferreds.job.promise.then((function(_this) {
+      return function(data) {
+        return $http.get("/jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
+          currentJob.exceptions = exceptions;
+          return deferred.resolve(exceptions);
+        });
+      };
+    })(this));
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
+  var refresh;
+  $scope.jobObserver = function() {
+    $scope.runningJobs = JobsService.getJobs('running');
+    return $scope.finishedJobs = JobsService.getJobs('finished');
+  };
+  JobsService.registerObserver($scope.jobObserver);
+  $scope.$on('$destroy', function() {
+    return JobsService.unRegisterObserver($scope.jobObserver);
+  });
+  $scope.jobObserver();
+  OverviewService.loadOverview().then(function(data) {
+    return $scope.overview = data;
+  });
+  refresh = $interval(function() {
+    return OverviewService.loadOverview().then(function(data) {
+      return $scope.overview = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  return $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+}]);
+
+angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  var overview;
+  overview = {};
+  this.loadOverview = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("/overview").success(function(data, status, headers, config) {
+      overview = data;
+      return deferred.resolve(data);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+angular.module('flinkApp').controller('TaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
+  var refresh;
+  TaskManagersService.loadManagers().then(function(data) {
+    return $scope.managers = data;
+  });
+  refresh = $interval(function() {
+    return TaskManagersService.loadManagers().then(function(data) {
+      return $scope.managers = data;
+    });
+  }, flinkConfig["refresh-interval"]);
+  $scope.$on('$destroy', function() {
+    return $interval.cancel(refresh);
+  });
+  $scope.managerId = null;
+  return $scope.changeManager = function(managerId) {
+    if (managerId !== $scope.managerId) {
+      return $scope.managerId = managerId;
+    } else {
+      return $scope.managerId = null;
+    }
+  };
+}]);
+
+angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
+  this.loadManagers = function() {
+    var deferred;
+    deferred = $q.defer();
+    $http.get("/taskmanagers").success(function(data, status, headers, config) {
+      return deferred.resolve(data['taskmanagers']);
+    });
+    return deferred.promise;
+  };
+  return this;
+}]);
+
+<<<<<<< HEAD:flink-runtime-web/web-dashboard/web/js/index.js
+//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2Vycy90YXNrbWF
 uYWdlcnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2Vycy90YXNrbWFuYWdlcnMuY3RybC5qcyIsIm1vZHVsZXMvdGFza21hbmFnZXJzL3Rhc2ttYW5hZ2Vycy5zdmMuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlcnMvdGFza21hbmFnZXJzLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQzNCSCxPRDRCQSxZQUFZLGFBQWEsS0FBSyxTQUFDLFFBQUQ7SUFDNUIsUUFBUSxPQUFPLGFBQWE7SUFFNUIsWUFBWTtJQzVCWixPRDhCQSxVQUFVLFdBQUE7TUM3QlIsT0Q4QkEsWUFBWTtPQUNaLFlBQVk7O0lBS2pCLGlDQUFPLFNBQUMsdUJBQUQ7RUNoQ04sT0RpQ0Esc0JBQXNCO0lBSXZCLGdEQUFPLFNBQUMsZ0JBQWdCLG9CQUFqQjtFQUNOLGVBQWUsTUFBTSxZQUNuQjtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxnQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxrQkFDTDtJ
 QUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxjQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxtQkFDTDtJQUFBLEtBQUs7SUFDTCxVQUFVO0lBQ1YsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sNEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdDQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxnQkFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx1QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLDhCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxRQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxnQkFDT
 DtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxjQUNIO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTs7O0tBRXBCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0scUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7Ozs7RUNmbkIsT0RpQkEsbUJBQW1CLFVBQVU7O0FDZi9CO0FDaEpBLFFBQVEsT0FBTyxZQUlkLFVBQVUsMkJBQVcsU0FBQyxhQUFEO0VDckJwQixPRHNCQTtJQUFBLFlBQVk7SUFDWixTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLGlCQUFpQixZQUFZLG9CQUFvQixNQUFNOzs7O0lBSTVELFVBQVUsb0NBQW9CLFNBQUMsYUFBRDtFQ3JCN0IsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLGVBQWU7TUFDZixRQUFROztJQUVWLFVBQVU7SUFFVixNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01DckJGLE9Ec0JGLE1BQU0sZ0JBQWdCLFdBQUE7UUNyQmxCLE9Ec0JGLHNDQUFzQyxZQUFZLG9CQUFvQixNQUFNOzs7O0lBSWpGLFVBQV
 UsaUJBQWlCLFdBQUE7RUNyQjFCLE9Ec0JBO0lBQUEsU0FBUztJQUNULE9BQ0U7TUFBQSxPQUFPOztJQUVULFVBQVU7OztBQ2xCWjtBQ3BCQSxRQUFRLE9BQU8sWUFFZCxPQUFPLG9EQUE0QixTQUFDLHFCQUFEO0VBQ2xDLElBQUE7RUFBQSxpQ0FBaUMsU0FBQyxPQUFPLFFBQVEsZ0JBQWhCO0lBQy9CLElBQWMsT0FBTyxVQUFTLGVBQWUsVUFBUyxNQUF0RDtNQUFBLE9BQU87O0lDaEJQLE9Ea0JBLE9BQU8sU0FBUyxPQUFPLFFBQVEsT0FBTyxnQkFBZ0I7TUFBRSxNQUFNOzs7RUFFaEUsK0JBQStCLFlBQVksb0JBQW9CO0VDZi9ELE9EaUJBO0lBRUQsT0FBTyxnQkFBZ0IsV0FBQTtFQ2pCdEIsT0RrQkEsU0FBQyxNQUFEO0lBRUUsSUFBRyxNQUFIO01DbEJFLE9Ea0JXLEtBQUssUUFBUSxTQUFTLEtBQUssUUFBUSxXQUFVO1dBQTFEO01DaEJFLE9EZ0JpRTs7O0dBRXRFLE9BQU8sU0FBUyxXQUFBO0VDZGYsT0RlQSxTQUFDLE9BQU8sV0FBUjtJQUNFLElBQUEsUUFBQTtJQUFBLElBQWUsTUFBTSxXQUFXLFdBQVcsQ0FBSSxTQUFTLFFBQXhEO01BQUEsT0FBTzs7SUFDUCxJQUFrQixPQUFPLGNBQWEsYUFBdEM7TUFBQSxZQUFZOztJQUNaLFFBQVEsQ0FBRSxTQUFTLE1BQU0sTUFBTSxNQUFNLE1BQU07SUFDM0MsU0FBUyxLQUFLLE1BQU0sS0FBSyxJQUFJLFNBQVMsS0FBSyxJQUFJO0lDVC9DLE9EVUEsQ0FBQyxRQUFRLEtBQUssSUFBSSxNQUFNLEtBQUssTUFBTSxVQUFVLFFBQVEsYUFBYSxNQUFNLE1BQU07OztBQ1BsRjtBQ2h
 CQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDhDQUFlLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ3RCLEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxXQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQ3BCUCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHNCQTs7QUNwQkY7QUNPQSxRQUFRLE9BQU8sWUFFZCxXQUFXLG9FQUE4QixTQUFDLFFBQVEseUJBQVQ7RUNuQnhDLE9Eb0JBLHdCQUF3QixhQUFhLEtBQUssU0FBQyxNQUFEO0lBQ3hDLElBQUksT0FBQSxjQUFBLE1BQUo7TUFDRSxPQUFPLGFBQWE7O0lDbEJ0QixPRG1CQSxPQUFPLFdBQVcsWUFBWTs7O0FDaEJsQztBQ1VBLFFBQVEsT0FBTyxZQUVkLFFBQVEsMERBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHNCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUNwQlQsT0RxQkEsU0FBUyxRQUFROztJQ25CbkIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7O0FDbkJGO0FDSUEsUUFBUSxPQUFPLFlBRWQsV0FBVyw2RUFBeUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUNuQyxPQUFPLGNBQWMsV0FBQTtJQ25CbkIsT0RvQkEsT0FBTyxPQUFPLFlBQVksUUFBUTs7RUFFcEMsWUFBWSxpQkFBaUIs
 T0FBTztFQUNwQyxPQUFPLElBQUksWUFBWSxXQUFBO0lDbkJyQixPRG9CQSxZQUFZLG1CQUFtQixPQUFPOztFQ2xCeEMsT0RvQkEsT0FBTztJQUlSLFdBQVcsK0VBQTJCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDckMsT0FBTyxjQUFjLFdBQUE7SUN0Qm5CLE9EdUJBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ3RCckIsT0R1QkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNyQnhDLE9EdUJBLE9BQU87SUFJUixXQUFXLHFIQUF1QixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQWEsWUFBWSxhQUFhLFdBQXJFO0VBQ2pDLElBQUE7RUFBQSxRQUFRLElBQUk7RUFFWixPQUFPLFFBQVEsYUFBYTtFQUM1QixPQUFPLE1BQU07RUFDYixPQUFPLE9BQU87RUFDZCxPQUFPLFdBQVc7RUFFbEIsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtJQUMzQyxPQUFPLE1BQU07SUFDYixPQUFPLE9BQU8sS0FBSztJQzFCbkIsT0QyQkEsT0FBTyxXQUFXLEtBQUs7O0VBRXpCLFlBQVksVUFBVSxXQUFBO0lDMUJwQixPRDJCQSxZQUFZLFFBQVEsYUFBYSxPQUFPLEtBQUssU0FBQyxNQUFEO01BQzNDLE9BQU8sTUFBTTtNQzFCYixPRDRCQSxPQUFPLFdBQVc7O0tBRXBCLFlBQVk7RUMzQmQsT0Q2QkEsT0FBTyxJQUFJLFlBQVksV0FBQTtJQUNyQixPQUFPLE1BQU07SUFDYixPQUFPLE9BQU87SUFDZCxPQUFPLFdBQVc7SUM1QmxCLE9EOEJBLFVBQVUsT
 0FBTzs7SUFLcEIsV0FBVyx5RUFBcUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUMvQixRQUFRLElBQUk7RUFFWixPQUFPLFNBQVM7RUFDaEIsT0FBTyxlQUFlO0VBQ3RCLE9BQU8sWUFBWSxZQUFZO0VBRS9CLE9BQU8sYUFBYSxTQUFDLFFBQUQ7SUFDbEIsSUFBRyxXQUFVLE9BQU8sUUFBcEI7TUFDRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxTQUFTO01BQ2hCLE9BQU8sV0FBVztNQUNsQixPQUFPLGVBQWU7TUNsQ3RCLE9Eb0NBLE9BQU8sV0FBVztXQU5wQjtNQVNFLE9BQU8sU0FBUztNQUNoQixPQUFPLGVBQWU7TUFDdEIsT0FBTyxTQUFTO01BQ2hCLE9BQU8sV0FBVztNQ3BDbEIsT0RxQ0EsT0FBTyxlQUFlOzs7RUFFMUIsT0FBTyxpQkFBaUIsV0FBQTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxlQUFlO0lBQ3RCLE9BQU8sU0FBUztJQUNoQixPQUFPLFdBQVc7SUNuQ2xCLE9Eb0NBLE9BQU8sZUFBZTs7RUNsQ3hCLE9Eb0NBLE9BQU8sYUFBYSxXQUFBO0lDbkNsQixPRG9DQSxPQUFPLGVBQWUsQ0FBQyxPQUFPOztJQUlqQyxXQUFXLHVEQUE2QixTQUFDLFFBQVEsYUFBVDtFQUN2QyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxLQUF2RDtJQUNFLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7TUN0QzFDLE9EdUNBLE9BQU8sV0FBVzs7O0VDcEN0QixPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0
 lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxZQUFZLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtRQ3JDMUMsT0RzQ0EsT0FBTyxXQUFXOzs7O0lBSXpCLFdBQVcsMkRBQWlDLFNBQUMsUUFBUSxhQUFUO0VBQzNDLFFBQVEsSUFBSTtFQUVaLElBQUcsT0FBTyxXQUFZLENBQUMsT0FBTyxVQUFVLENBQUMsT0FBTyxPQUFPLGVBQXZEO0lBQ0UsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01BQzlDLE9BQU8sZUFBZSxLQUFLO01DdEMzQixPRHVDQSxPQUFPLHNCQUFzQixLQUFLOzs7RUNwQ3RDLE9Ec0NBLE9BQU8sSUFBSSxVQUFVLFNBQUMsT0FBRDtJQUNuQixRQUFRLElBQUk7SUFDWixJQUFHLE9BQU8sUUFBVjtNQ3JDRSxPRHNDQSxZQUFZLGdCQUFnQixPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUFDOUMsT0FBTyxlQUFlLEtBQUs7UUNyQzNCLE9Ec0NBLE9BQU8sc0JBQXNCLEtBQUs7Ozs7SUFJekMsV0FBVyxtRkFBK0IsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUEvQjtFQUN6QyxRQUFRLElBQUk7RUFFWixZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO0lDdENoRCxPRHVDQSxPQUFPLFNBQVM7O0VDckNsQixPRHVDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lDdENaLE9EdUNBLFlBQVksVUFBVSxhQUFhLFVBQVUsS0FBSyxTQUFDLE1BQUQ7TUN0Q2hELE9EdUNBLE9BQU8sU0FBUzs7O0lBSXJCLFdBQVcsK0VBQTJCLFNBQUMsUUFBUSx
 RQUFRLGNBQWMsYUFBL0I7RUN2Q3JDLE9Ed0NBLFlBQVksaUJBQWlCLEtBQUssU0FBQyxNQUFEO0lDdkNoQyxPRHdDQSxPQUFPLGFBQWE7O0lBSXZCLFdBQVcscURBQTJCLFNBQUMsUUFBUSxhQUFUO0VBQ3JDLFFBQVEsSUFBSTtFQ3pDWixPRDJDQSxPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01DMUNoQixPRDRDQSxZQUFZLFFBQVEsUUFBUSxLQUFLLFNBQUMsTUFBRDtRQzNDL0IsT0Q0Q0EsT0FBTyxPQUFPOztXQUpsQjtNQU9FLE9BQU8sU0FBUztNQzNDaEIsT0Q0Q0EsT0FBTyxPQUFPOzs7O0FDeENwQjtBQ25IQSxRQUFRLE9BQU8sWUFJZCxVQUFVLHFCQUFVLFNBQUMsUUFBRDtFQ3JCbkIsT0RzQkE7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLE1BQU07O0lBRVIsTUFBTSxTQUFDLE9BQU8sTUFBTSxPQUFkO01BQ0osSUFBQSxhQUFBLFlBQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxLQUFLLFVBQVUsU0FBQyxTQUFTLEdBQVY7VUFDN0IsSUFBQTtVQUFBLFFBQVE7WUFDTjtjQUNFLE9BQU87Y0FDUCxPQUFPO2NBQ1AsYUFBYTtjQUNiLGVBQWUsUUFBUSxXQUFXO2NBQ2xDLGFBQWEsUUFBUSxXQUFXO2NB
 Q2hDLE1BQU07ZUFFUjtjQUNFLE9BQU87Y0FDUCxPQUFPO2NBQ1AsYUFBYTtjQUNiLGVBQWUsUUFBUSxXQUFXO2NBQ2xDLGFBQWEsUUFBUSxXQUFXO2NBQ2hDLE1BQU07OztVQUlWLElBQUcsUUFBUSxXQUFXLGNBQWMsR0FBcEM7WUFDRSxNQUFNLEtBQUs7Y0FDVCxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUN0QlIsT0R5QkYsU0FBUyxLQUFLO1lBQ1osT0FBTyxNQUFJLFFBQVEsVUFBUSxPQUFJLFFBQVE7WUFDdkMsT0FBTzs7O1FBR1gsUUFBUSxHQUFHLFdBQVcsUUFDckIsV0FBVztVQUNWLFFBQVEsR0FBRyxLQUFLLE9BQU87VUFFdkIsVUFBVTtXQUVYLE9BQU8sVUFDUCxZQUFZLFNBQUMsT0FBRDtVQzVCVCxPRDZCRjtXQUVELE9BQU87VUFBRSxNQUFNO1VBQUssT0FBTztVQUFHLEtBQUs7VUFBRyxRQUFRO1dBQzlDLFdBQVcsSUFDWDtRQzFCQyxPRDRCRixNQUFNLEdBQUcsT0FBTyxPQUNmLE1BQU0sVUFDTixLQUFLOztNQUVSLFlBQVksTUFBTTs7O0lBTXJCLFVBQVUsdUJBQVksU0FBQyxRQUFEO0VDaENyQixPRGlDQTtJQUFBLFVBQVU7SUFFVixPQUNFO01BQUEsVUFBVTtNQUNWLE9BQU87O0lBRVQsTUFBTSxTQUFDLE9BQU8sTUFBTSxPQUFkO01BQ0osSUFBQSxhQUFBLFlBQUEsT0FBQTtNQUFBLFFBQVEsS0FBSyxXQUFXO01BRXhCLGFBQWEsS0FBSztNQUNsQixRQUFRLFFBQVEsT0FBTyxLQUFLLFNBQVM7TUFFckMsa
 UJBQWlCLFNBQUMsT0FBRDtRQ2pDYixPRGtDRixNQUFNLFFBQVEsUUFBUTs7TUFFeEIsY0FBYyxTQUFDLE1BQUQ7UUFDWixJQUFBLE9BQUEsS0FBQTtRQUFBLEdBQUcsT0FBTyxPQUFPLFVBQVUsS0FBSztRQUVoQyxXQUFXO1FBRVgsUUFBUSxRQUFRLE1BQU0sU0FBQyxRQUFEO1VBQ3BCLElBQUcsT0FBTyxnQkFBZ0IsQ0FBQyxHQUEzQjtZQUNFLElBQUcsT0FBTyxTQUFRLGFBQWxCO2NDbENJLE9EbUNGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29CQUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTzs7OzttQkFSbkI7Y0NyQkksT0RnQ0YsU0FBUyxLQUNQO2dCQUFBLE9BQU87a0JBQ0w7b0JBQUEsT0FBTyxlQUFlLE9BQU87b0JBQzdCLE9BQU87b0JBQ1AsYUFBYTtvQkFDYixlQUFlLE9BQU87b0JBQ3RCLGFBQWEsT0FBTztvQkFDcEIsTUFBTSxPQUFPO29CQUNiLE1BQU0sT0FBTzs7Ozs7OztRQUd2QixRQUFRLEdBQUcsV0FBVyxRQUFRLE1BQU0sU0FBQyxHQUFHLEdBQUcsT0FBUDtVQUNsQyxJQUFHLEVBQUUsTUFBTDtZQzFCSSxPRDJCRixPQUFPLEdBQUcsOEJBQThCO2NBQUUsT0FBTyxNQUFNO2NBQU8sVUFBVSxFQUFFOzs7V0FHN0UsV0FBVztVQUNWLFFBQVEsR0FBRyxLQUFLLE9BQU87VUFHdkIsVUFBVTtXQUVYLE9BQU8sUUFDUCxPQUFPO1VBQUUsTUFBTTtVQUFHLE9BQU87VUFBRyxLQUFLO1VBQU
 csUUFBUTtXQUM1QyxXQUFXLElBQ1gsaUJBQ0E7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixNQUFNLE9BQU8sTUFBTSxVQUFVLFNBQUMsTUFBRDtRQUMzQixJQUFxQixNQUFyQjtVQzdCSSxPRDZCSixZQUFZOzs7OztJQU1qQixVQUFVLHdCQUFXLFNBQUMsVUFBRDtFQzdCcEIsT0Q4QkE7SUFBQSxVQUFVO0lBUVYsT0FDRTtNQUFBLE1BQU07TUFDTixTQUFTOztJQUVYLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsWUFBQSxZQUFBLGlCQUFBLGlCQUFBLFlBQUEsV0FBQSxZQUFBLFVBQUEsV0FBQSw2QkFBQSxHQUFBLGFBQUEsd0JBQUEsT0FBQSxpQkFBQSxPQUFBLGdCQUFBLGdCQUFBLFVBQUEsZUFBQSxlQUFBO01BQUEsSUFBSTtNQUNKLFdBQVcsR0FBRyxTQUFTO01BQ3ZCLFlBQVk7TUFDWixRQUFRLE1BQU07TUFFZCxpQkFBaUIsS0FBSyxXQUFXO01BQ2pDLFFBQVEsS0FBSyxXQUFXLFdBQVc7TUFDbkMsaUJBQWlCLEtBQUssV0FBVztNQUVqQyxZQUFZLEdBQUcsT0FBTztNQUN0QixhQUFhLEdBQUcsT0FBTztNQUN2QixXQUFXLEdBQUcsT0FBTztNQUtyQixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLEtBQUssV0FBVyxJQUFJLE1BQU07TUFFMUMsTUFBTSxTQUFTLFdBQUE7UUFDYixJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsWUFBWSxTQUFTO1VBQ3JCLEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM
 7VUFDeEQsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFNBQVMsVUFBVSxDQUFFLElBQUk7VUMxQ3ZCLE9ENkNGLFdBQVcsS0FBSyxhQUFhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUVoRyxNQUFNLFVBQVUsV0FBQTtRQUNkLElBQUEsV0FBQSxJQUFBO1FBQUEsSUFBRyxTQUFTLFVBQVUsTUFBdEI7VUFHRSxTQUFTLE1BQU0sU0FBUyxVQUFVO1VBQ2xDLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxVQUFVLENBQUUsSUFBSTtVQzVDdkIsT0QrQ0YsV0FBVyxLQUFLLGFBQWEsZUFBZSxLQUFLLE1BQU0sS0FBSyxhQUFhLFNBQVMsVUFBVTs7O01BR2hHLGtCQUFrQixTQUFDLElBQUQ7UUFDaEIsSUFBQTtRQUFBLGFBQWE7UUFDYixJQUFHLENBQUEsR0FBQSxpQkFBQSxVQUFxQixHQUFBLGtCQUFBLE9BQXhCO1VBQ0UsY0FBYztVQUNkLElBQW1DLEdBQUEsaUJBQUEsTUFBbkM7WUFBQSxjQUFjLEdBQUc7O1VBQ2pCLElBQWdELEdBQUcsY0FBYSxXQUFoRTtZQUFBLGNBQWMsT0FBTyxHQUFHLFlBQVk7O1VBQ3BDLElBQWtELEdBQUcsbUJBQWtCLFdBQXZFO1lBQUEsY0FBYyxVQUFVLEdBQUc7O1VBQzNCLGNBQWM7O1FDdENkLE9EdUNGOztNQUlGLHlCQUF5QixTQUFDLE1BQUQ7UUN4Q3JCLE9E
 eUNELFNBQVEscUJBQXFCLFNBQVEseUJBQXlCLFNBQVEsYUFBYSxTQUFRLGlCQUFpQixTQUFRLGlCQUFpQixTQUFROztNQUVoSixjQUFjLFNBQUMsSUFBSSxNQUFMO1FBQ1osSUFBRyxTQUFRLFVBQVg7VUN4Q0ksT0R5Q0Y7ZUFFRyxJQUFHLHVCQUF1QixPQUExQjtVQ3pDRCxPRDBDRjtlQURHO1VDdkNELE9EMkNBOzs7TUFHTixrQkFBa0IsU0FBQyxJQUFJLE1BQU0sTUFBTSxNQUFqQjtRQUVoQixJQUFBLFlBQUE7UUFBQSxhQUFhLHVCQUF1QixRQUFRLGFBQWEsR0FBRyxLQUFLLHlCQUF5QixZQUFZLElBQUksUUFBUTtRQUdsSCxJQUFHLFNBQVEsVUFBWDtVQUNFLGNBQWMscUNBQXFDLEdBQUcsV0FBVztlQURuRTtVQUdFLGNBQWMsMkJBQTJCLEdBQUcsV0FBVzs7UUFDekQsSUFBRyxHQUFHLGdCQUFlLElBQXJCO1VBQ0UsY0FBYztlQURoQjtVQUdFLFdBQVcsR0FBRztVQUdkLFdBQVcsY0FBYztVQUN6QixjQUFjLDJCQUEyQixXQUFXOztRQUd0RCxJQUFHLEdBQUEsaUJBQUEsTUFBSDtVQUNFLGNBQWMsNEJBQTRCLEdBQUcsSUFBSSxNQUFNO2VBRHpEO1VBS0UsSUFBK0MsdUJBQXVCLE9BQXRFO1lBQUEsY0FBYyxTQUFTLE9BQU87O1VBQzlCLElBQXFFLEdBQUcsZ0JBQWUsSUFBdkY7WUFBQSxjQUFjLHNCQUFzQixHQUFHLGNBQWM7O1VBQ3JELElBQXdGLEdBQUcsYUFBWSxXQUF2RztZQUFBLGNBQWMsb0JBQW9CLGNBQWMsR0FBRyxxQkFBcUI7OztRQUcxRSxjQUFjO1FDM0NaLE9ENENGOztNQUdGLDhCQUE4QixTQUFDLElBQ
 UksTUFBTSxNQUFYO1FBQzVCLElBQUEsWUFBQTtRQUFBLFFBQVEsU0FBUztRQUVqQixhQUFhLGlCQUFpQixRQUFRLGFBQWEsT0FBTyxhQUFhLE9BQU87UUM1QzVFLE9ENkNGOztNQUdGLGdCQUFnQixTQUFDLEdBQUQ7UUFFZCxJQUFBO1FBQUEsSUFBRyxFQUFFLE9BQU8sT0FBTSxLQUFsQjtVQUNFLElBQUksRUFBRSxRQUFRLEtBQUs7VUFDbkIsSUFBSSxFQUFFLFFBQVEsS0FBSzs7UUFDckIsTUFBTTtRQUNOLE9BQU0sRUFBRSxTQUFTLElBQWpCO1VBQ0UsTUFBTSxNQUFNLEVBQUUsVUFBVSxHQUFHLE1BQU07VUFDakMsSUFBSSxFQUFFLFVBQVUsSUFBSSxFQUFFOztRQUN4QixNQUFNLE1BQU07UUMzQ1YsT0Q0Q0Y7O01BRUYsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLFVBQWtCLE1BQU0sTUFBdEM7UUMzQ1QsSUFBSSxZQUFZLE1BQU07VUQyQ0MsV0FBVzs7UUFFcEMsSUFBRyxHQUFHLE9BQU0sS0FBSyxrQkFBakI7VUN6Q0ksT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksbUJBQW1CLE1BQU07WUFDcEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLHVCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSx1QkFBdUIsTUFBTTtZQUN4RCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssU0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQU
 csSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksV0FBVyxNQUFNO1lBQzVDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGVBQWUsTUFBTTtZQUNoRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssZ0JBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLGlCQUFpQixNQUFNO1lBQ2xELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFKdEI7VUNuQ0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksSUFBSSxNQUFNO1lBQ3JDLFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7OztNQUU3QixhQUFhLFNBQUMsR0FBRyxNQUFNLElBQUksZUFBZSxNQUFNLGNBQW5DO1FBQ1gsSUFBQTtRQUFBLElBQU8sY0FBYyxRQUFRLEtBQUssUUFBTyxDQUFDLEdBQTFDO1VDdENJLE9EdUNGLEVBQUUsUUFBUSxLQUFLLElBQUksR0FBRyxJQUNwQjtZQUFBLE9BQU8sZ0JBQWdCO1lBQ3ZCLFdBQVc7WUFDWCxXQUFXOztlQUpmO1V
 BT0UsY0FBYyxjQUFjLE1BQU0sS0FBSztVQUV2QyxJQUFBLEVBQU8sQ0FBQyxlQUFlLGFBQWEsUUFBUSxZQUFZLE1BQU0sQ0FBQyxJQUEvRDtZQUNFLGFBQWEsS0FBSyxZQUFZO1lBQzlCLEVBQUUsUUFBUSxZQUFZLElBQ3BCO2NBQUEsT0FBTyxnQkFBZ0IsYUFBYTtjQUNwQyxXQUFXO2NBQ1gsU0FBTyxZQUFZLGFBQWE7O1lDdENoQyxPRHdDRixFQUFFLFFBQVEsWUFBWSxJQUFJLEdBQUcsSUFDM0I7Y0FBQSxPQUFPLGdCQUFnQjtjQUN2QixXQUFXOzs7OztNQUVuQixrQkFBa0IsU0FBQyxHQUFHLE1BQUo7UUFDaEIsSUFBQSxJQUFBLGVBQUEsVUFBQSxHQUFBLEdBQUEsS0FBQSxNQUFBLE1BQUEsTUFBQSxjQUFBLE1BQUEsR0FBQSxLQUFBLElBQUE7UUFBQSxnQkFBZ0I7UUFDaEIsZUFBZTtRQUVmLElBQUcsS0FBQSxTQUFBLE1BQUg7VUFFRSxZQUFZLEtBQUs7ZUFGbkI7VUFNRSxZQUFZLEtBQUs7VUFDakIsV0FBVzs7UUFFYixLQUFBLElBQUEsR0FBQSxNQUFBLFVBQUEsUUFBQSxJQUFBLEtBQUEsS0FBQTtVQ3ZDSSxLQUFLLFVBQVU7VUR3Q2pCLE9BQU87VUFDUCxPQUFPO1VBRVAsSUFBRyxHQUFHLGVBQU47WUFDRSxLQUFTLElBQUEsUUFBUSxTQUFTLE1BQU07Y0FBRSxZQUFZO2NBQU0sVUFBVTtlQUFRLFNBQVM7Y0FDN0UsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTOztZQUdYLFVBQVUsR0FBRyxNQUFNO1lBRW5CLGdCQUFnQixJQUFJO1lBRXBCLElBQVEsSUFBQSxR
 QUFRO1lBQ2hCLFNBQVMsT0FBTyxLQUFLLEtBQUssR0FBRztZQUM3QixPQUFPLEdBQUcsUUFBUTtZQUNsQixPQUFPLEdBQUcsUUFBUTtZQUVsQixRQUFRLFFBQVEsZ0JBQWdCOztVQUVsQyxXQUFXLEdBQUcsTUFBTSxJQUFJLFVBQVUsTUFBTTtVQUV4QyxjQUFjLEtBQUssR0FBRztVQUd0QixJQUFHLEdBQUEsVUFBQSxNQUFIO1lBQ0UsTUFBQSxHQUFBO1lBQUEsS0FBQSxJQUFBLEdBQUEsT0FBQSxJQUFBLFFBQUEsSUFBQSxNQUFBLEtBQUE7Y0MxQ0ksT0FBTyxJQUFJO2NEMkNiLFdBQVcsR0FBRyxNQUFNLElBQUksZUFBZSxNQUFNOzs7O1FDdENqRCxPRHdDRjs7TUFHRixnQkFBZ0IsU0FBQyxNQUFNLFFBQVA7UUFDZCxJQUFBLElBQUEsR0FBQTtRQUFBLEtBQUEsS0FBQSxLQUFBLE9BQUE7VUFDRSxLQUFLLEtBQUssTUFBTTtVQUNoQixJQUFjLEdBQUcsT0FBTSxRQUF2QjtZQUFBLE9BQU87O1VBR1AsSUFBRyxHQUFBLGlCQUFBLE1BQUg7WUFDRSxLQUFBLEtBQUEsR0FBQSxlQUFBO2NBQ0UsSUFBK0IsR0FBRyxjQUFjLEdBQUcsT0FBTSxRQUF6RDtnQkFBQSxPQUFPLEdBQUcsY0FBYzs7Ozs7O01BRWhDLFlBQVksU0FBQyxNQUFEO1FBQ1YsSUFBQSxHQUFBLFVBQUEsVUFBQSxJQUFBLGVBQUE7UUFBQSxJQUFRLElBQUEsUUFBUSxTQUFTLE1BQU07VUFBRSxZQUFZO1VBQU0sVUFBVTtXQUFRLFNBQVM7VUFDNUUsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTOztRQUdYLGdCQ
 UFnQixHQUFHO1FBRW5CLFdBQWUsSUFBQSxRQUFRO1FBQ3ZCLFdBQVcsS0FBSyxVQUFVO1FBRTFCLEtBQUEsS0FBQSxXQUFBO1VDakNJLEtBQUssVUFBVTtVRGtDakIsVUFBVSxPQUFPLGFBQWEsSUFBSSxNQUFNLEtBQUssVUFBVTs7UUFFekQsV0FBVztRQUVYLGdCQUFnQixLQUFLLE1BQU0sQ0FBQyxRQUFRLFFBQVEsZ0JBQWdCLFVBQVUsRUFBRSxRQUFRLFFBQVEsWUFBWTtRQUNwRyxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixXQUFXLEVBQUUsUUFBUSxTQUFTLFlBQVk7UUFFdEcsU0FBUyxNQUFNLFVBQVUsVUFBVSxDQUFDLGVBQWU7UUFFbkQsV0FBVyxLQUFLLGFBQWEsZUFBZSxnQkFBZ0IsT0FBTyxnQkFBZ0IsYUFBYSxTQUFTLFVBQVU7UUFFbkgsU0FBUyxHQUFHLFFBQVEsV0FBQTtVQUNsQixJQUFBO1VBQUEsS0FBSyxHQUFHO1VDbkNOLE9Eb0NGLFdBQVcsS0FBSyxhQUFhLGVBQWUsR0FBRyxZQUFZLGFBQWEsR0FBRyxRQUFROztRQUVyRixTQUFTO1FDbkNQLE9EcUNGLFdBQVcsVUFBVSxTQUFTLEdBQUcsU0FBUyxTQUFDLEdBQUQ7VUNwQ3RDLE9EcUNGLE1BQU0sUUFBUTtZQUFFLFFBQVE7Ozs7TUFFNUIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLFNBQUQ7UUFDdkIsSUFBc0IsU0FBdEI7VUNqQ0ksT0RpQ0osVUFBVTs7Ozs7O0FDM0JoQjtBQzFhQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDhFQUFlLFNBQUMsT0FBTyxhQUFhLE1BQU0sVUFBVSxJQUFJLFVBQXpDO0VBQ3RCLElBQUEsWUFBQS
 xhQUFBLFdBQUEsY0FBQSxNQUFBO0VBQUEsYUFBYTtFQUNiLGNBQWM7RUFFZCxZQUFZO0VBQ1osT0FBTztJQUNMLFNBQVM7SUFDVCxVQUFVO0lBQ1YsV0FBVztJQUNYLFFBQVE7O0VBR1YsZUFBZTtFQUVmLGtCQUFrQixXQUFBO0lDckJoQixPRHNCQSxRQUFRLFFBQVEsY0FBYyxTQUFDLFVBQUQ7TUNyQjVCLE9Ec0JBOzs7RUFFSixLQUFDLG1CQUFtQixTQUFDLFVBQUQ7SUNwQmxCLE9EcUJBLGFBQWEsS0FBSzs7RUFFcEIsS0FBQyxxQkFBcUIsU0FBQyxVQUFEO0lBQ3BCLElBQUE7SUFBQSxRQUFRLGFBQWEsUUFBUTtJQ25CN0IsT0RvQkEsYUFBYSxPQUFPLE9BQU87O0VBRTdCLEtBQUMsWUFBWSxXQUFBO0lDbkJYLE9Eb0JBLENBRUUsYUFDQSxhQUNBLFdBQ0EsWUFDQSxVQUNBLGFBQ0E7O0VBR0osS0FBQyxzQkFBc0IsU0FBQyxPQUFEO0lBQ3JCLFFBQU8sTUFBTTtNQUFiLEtBQ087UUM1QkgsT0Q0Qm1CO01BRHZCLEtBRU87UUMzQkgsT0QyQmlCO01BRnJCLEtBR087UUMxQkgsT0QwQm9CO01BSHhCLEtBSU87UUN6QkgsT0R5Qm9CO01BSnhCLEtBS087UUN4QkgsT0R3QmtCO01BTHRCLEtBTU87UUN2QkgsT0R1Qm9CO01BTnhCLEtBT087UUN0QkgsT0RzQmtCO01BUHRCLEtBUU87UUNyQkgsT0RxQmdCO01BUnBCO1FDWEksT0RvQkc7OztFQUVULEtBQUMsY0FBYyxTQUFDLE1BQUQ7SUNsQmIsT0RtQkEsUUFBUSxRQUFRLE1BQU0sU0FBQyxNQUFNLFFBQVA7TUFDcEIsSUFBQSxFQUFPLEtBQUssY0FBYyxDQUFDLElBQTNCO1F
 DbEJFLE9EbUJBLEtBQUssY0FBYyxLQUFLLGdCQUFnQixLQUFLOzs7O0VBRW5ELEtBQUMsa0JBQWtCLFNBQUMsTUFBRDtJQUNqQixRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsUUFBUSxHQUFUO01DaEI3QixPRGlCQSxPQUFPLE9BQU87O0lDZmhCLE9EaUJBLEtBQUssU0FBUyxRQUFRO01BQ3BCLE1BQU07TUFDTixjQUFjLEtBQUssV0FBVztNQUM5QixZQUFZLEtBQUssV0FBVyxhQUFhO01BQ3pDLE1BQU07OztFQUdWLEtBQUMsV0FBVyxXQUFBO0lBQ1YsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxnQkFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lB
 RW5CLE1BQU0sSUFBSSxXQUFXLE9BQ3BCLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFdBQVcsUUFBUSxXQUM1QixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQ
 yxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxXQUFXLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQy9ELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RFRixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsVUFDcEQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSS
 xRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDL0QsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFdBQVcsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDL0QsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ESnlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxXQUFXLFdBQVcsTUFBTSxlQUNyQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVk
 sV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLGFBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLHdGQUEwQixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDcEMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VBRWQsT0FBTyxJQUFJLFlBQVks
 V0FBQTtJQ2xCckIsT0RtQkEsVUFBVSxPQUFPOztFQUVuQixPQUFPLFlBQVk7RUNsQm5CLE9Eb0JBLE9BQU8sZ0JBQWdCLFNBQUMsV0FBRDtJQUNyQixJQUFHLGNBQWEsT0FBTyxXQUF2QjtNQ25CRSxPRG9CQSxPQUFPLFlBQVk7V0FEckI7TUNqQkUsT0RvQkEsT0FBTyxZQUFZOzs7O0FDaEJ6QjtBQ0pBLFFBQVEsT0FBTyxZQUVkLFFBQVEsc0RBQXVCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzlCLEtBQUMsZUFBZSxXQUFBO0lBQ2QsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxpQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUNwQlAsT0RxQkEsU0FBUyxRQUFRLEtBQUs7O0lDbkJ4QixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkYiLCJmaWxlIjoiaW5kZXguanMiLCJzb3VyY2VzQ29udGVudCI6WyIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgb
 m90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKCRyb290U2NvcGUpIC0+XG4gICRyb290U2NvcGUuc2lkZWJhclZpc2libGUgPSBmYWxzZVxuICAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gLT5cbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcbiAgICAkcm
 9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93J1xuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi52YWx1ZSAnZmxpbmtDb25maWcnLCB7XG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxufVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5ydW4gKEpvYnNTZXJ2aWNlLCBNYWluU2VydmljZSwgZmxpbmtDb25maWcsICRpbnRlcnZhbCkgLT5cbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cbiAgICBhbmd1bGFyLmV4dGVuZCBmbGlua0NvbmZpZywgY29uZmlnXG5cbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXG5cbiAgICAkaW50ZXJ2YWwgLT5cbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcbiAgICAsIGZsaW5rQ29uZmlnW1wicmVmcmVzaC1pbnRlcnZhbFwiXVxuXG5cbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxuICAkdWlWaWV3U2Nyb2xsUHJvdmlkZXIudXNlQW5jaG9yU2Nyb2xsKClcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxuXG4uY29uZmlnICgkc3RhdGVQcm92aWRlciwgJHVybFJvdXRlclByb3ZpZGVyKSAtPlxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXG4gICAgdXJsOiBcIi9vdmVydmlld1wiXG4gICAgdmlld3M
 6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9vdmVydmlldy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ092ZXJ2aWV3Q29udHJvbGxlcidcblxuICAuc3RhdGUgXCJydW5uaW5nLWpvYnNcIixcbiAgICB1cmw6IFwiL3J1bm5pbmctam9ic1wiXG4gICAgdmlld3M6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL3J1bm5pbmctam9icy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcbiAgXG4gIC5zdGF0ZSBcImNvbXBsZXRlZC1qb2JzXCIsXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXG4gICAgdmlld3M6XG4gICAgICBtYWluOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnQ29tcGxldGVkSm9ic0NvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxuICAgIHVybDogXCIvam9icy97am9iaWR9XCJcbiAgICBhYnN0cmFjdDogdHJ1ZVxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IucGxhblwiLFxuICAgIHVybDogXCJcIlxuICAgIGFic3RyYWN0OiB0cnVl
 XG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkNvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLm92ZXJ2aWV3XCIsXG4gICAgdXJsOiBcIlwiXG4gICAgdmlld3M6XG4gICAgICAnbm9kZS1kZXRhaWxzJzpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3Qub3ZlcnZpZXcuaHRtbFwiXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4uYWNjdW11bGF0b3JzXCIsXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxuICAgIHZpZXdzOlxuICAgICAgJ25vZGUtZGV0YWlscyc6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJyBcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXG4gICAgdXJsOiBcIi90aW1lbGluZVwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUta
 m9iLnRpbWVsaW5lLnZlcnRleFwiLFxuICAgIHVybDogXCIve3ZlcnRleElkfVwiXG4gICAgdmlld3M6XG4gICAgICB2ZXJ0ZXg6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcidcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnN0YXRpc3RpY3NcIixcbiAgICB1cmw6IFwiL3N0YXRpc3RpY3NcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcblxuICAuc3RhdGUgXCJzaW5nbGUtam9iLmV4Y2VwdGlvbnNcIixcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuZXhjZXB0aW9ucy5odG1sXCJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xuXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IucHJvcGVydGllc1wiLFxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUHJvcGVydGllc0NvbnRyb2xsZXInXG5cbi
 AgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcbiAgICB1cmw6IFwiL2NvbmZpZ1wiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5jb25maWcuaHRtbFwiXG5cbiAgLnN0YXRlIFwidGFza21hbmFnZXJzXCIsXG4gICAgdXJsOiBcIi90YXNrbWFuYWdlcnNcIlxuICAgIHZpZXdzOlxuICAgICAgbWFpbjpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvdGFza21hbmFnZXJzL2luZGV4Lmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnVGFza01hbmFnZXJzQ29udHJvbGxlcidcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyXCIsXG4gICAgICB1cmw6IFwiL2pvYm1hbmFnZXJcIlxuICAgICAgdmlld3M6XG4gICAgICAgIG1haW46XG4gICAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9pbmRleC5odG1sXCJcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyLmNvbmZpZ1wiLFxuICAgIHVybDogXCIvY29uZmlnXCJcbiAgICB2aWV3czpcbiAgICAgIGRldGFpbHM6XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvY29uZmlnLmh0bWxcIlxuICAgICAgICBjb250cm9sbGVyOiAnSm9iTWFuYWdlckNvbmZpZ0NvbnRyb2xsZXInXG5cbiAgLnN0YXRlIFwiam9ibWFuYWdlci5zdGRvdXRcIixcbiAgICB1cmw6IFwiL3N0ZG91dFwiXG4gICAgdmlld3M6XG4gICAgICBkZXRhaWxzOlx
 uICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL3N0ZG91dC5odG1sXCJcblxuICAuc3RhdGUgXCJqb2JtYW5hZ2VyLmxvZ2ZpbGVcIixcbiAgICB1cmw6IFwiL2xvZ2ZpbGVcIlxuICAgIHZpZXdzOlxuICAgICAgZGV0YWlsczpcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9sb2dmaWxlLmh0bWxcIlxuXG4gICR1cmxSb3V0ZXJQcm92aWRlci5vdGhlcndpc2UgXCIvb3ZlcnZpZXdcIlxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKS5ydW4oZnVuY3Rpb24oJHJvb3RTY29wZSkge1xuICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gZmFsc2U7XG4gIHJldHVybiAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gZnVuY3Rpb24oKSB7XG4gICAgJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZSA9ICEkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlO1xuICAgIHJldHVybiAkcm9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93JztcbiAgfTtcbn0pLnZhbHVlKCdmbGlua0NvbmZpZycsIHtcbiAgXCJyZWZyZXNoLWludGVydmFsXCI6IDEwMDAwXG59KS5ydW4oZnVuY3Rpb24oSm9ic1NlcnZpY2UsIE1haW5TZXJ2aWNlLCBmbGlua0NvbmZpZywgJGludGVydmFsKSB7XG4gIHJldHVybiBNYWluU2VydmljZS5sb2FkQ29uZmlnKCkudGhlbihmdW5jdGlvbihjb25maWcpIHtcbiAgICBhbmd1bGFyLmV4
 dGVuZChmbGlua0NvbmZpZywgY29uZmlnKTtcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpO1xuICAgIHJldHVybiAkaW50ZXJ2YWwoZnVuY3Rpb24oKSB7XG4gICAgICByZXR1cm4gSm9ic1NlcnZpY2UubGlzdEpvYnMoKTtcbiAgICB9LCBmbGlua0NvbmZpZ1tcInJlZnJlc2gtaW50ZXJ2YWxcIl0pO1xuICB9KTtcbn0pLmNvbmZpZyhmdW5jdGlvbigkdWlWaWV3U2Nyb2xsUHJvdmlkZXIpIHtcbiAgcmV0dXJuICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKTtcbn0pLmNvbmZpZyhmdW5jdGlvbigkc3RhdGVQcm92aWRlciwgJHVybFJvdXRlclByb3ZpZGVyKSB7XG4gICRzdGF0ZVByb3ZpZGVyLnN0YXRlKFwib3ZlcnZpZXdcIiwge1xuICAgIHVybDogXCIvb3ZlcnZpZXdcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9vdmVydmlldy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInJ1bm5pbmctam9ic1wiLCB7XG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL3J1bm5pbmctam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdSdW5uaW5nSm9ic0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuI
 CB9KS5zdGF0ZShcImNvbXBsZXRlZC1qb2JzXCIsIHtcbiAgICB1cmw6IFwiL2NvbXBsZXRlZC1qb2JzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIG1haW46IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9jb21wbGV0ZWQtam9icy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYlwiLCB7XG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIixcbiAgICBhYnN0cmFjdDogdHJ1ZSxcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW5cIiwge1xuICAgIHVybDogXCJcIixcbiAgICBhYnN0cmFjdDogdHJ1ZSxcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIiwge1xuICAgIHVybDogXCJcIixcbiAgICB2aWV3czoge1xuICAgICAgJ25vZGUtZGV0YWlscyc6IH
 tcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3Qub3ZlcnZpZXcuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbk92ZXJ2aWV3Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLCB7XG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIixcbiAgICB2aWV3czoge1xuICAgICAgJ25vZGUtZGV0YWlscyc6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucGxhbi5ub2RlLWxpc3QuYWNjdW11bGF0b3JzLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsIHtcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IudGltZWxpbmUudmVydGV4XCIsIHtcbiAgICB1cmw6IFwiL3t2ZXJ0ZXhJZH1cIixcbiAgICB2aWV3czoge1xuICAgICAgdmVydGV4OiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLnZlcnRleC5odG1sXCIsXG4gICA
 gICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLCB7XG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2Iuc3RhdGlzdGljcy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5leGNlcHRpb25zXCIsIHtcbiAgICB1cmw6IFwiL2V4Y2VwdGlvbnNcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5leGNlcHRpb25zLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIiwge1xuICAgIHVybDogXCIvcHJvcGVydGllc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnByb3BlcnRpZXMuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUHJvcGVydGllc0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IuY29uZmlnXCIsIHtcbiAgICB1cmw6IFwiL2NvbmZpZ1wiLFxu
 ICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwidGFza21hbmFnZXJzXCIsIHtcbiAgICB1cmw6IFwiL3Rhc2ttYW5hZ2Vyc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2Vycy9pbmRleC5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJqb2JtYW5hZ2VyXCIsIHtcbiAgICB1cmw6IFwiL2pvYm1hbmFnZXJcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL2luZGV4Lmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJqb2JtYW5hZ2VyLmNvbmZpZ1wiLCB7XG4gICAgdXJsOiBcIi9jb25maWdcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL2NvbmZpZy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyQ29uZmlnQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiam9ibWFuYWdlci5zdGRvdXRcIiwge1xuICAgIHVybDogXCIvc3Rkb3V0X
 CIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9zdGRvdXQuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcImpvYm1hbmFnZXIubG9nZmlsZVwiLCB7XG4gICAgdXJsOiBcIi9sb2dmaWxlXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9sb2dmaWxlLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSk7XG4gIHJldHVybiAkdXJsUm91dGVyUHJvdmlkZXIub3RoZXJ3aXNlKFwiL292ZXJ2aWV3XCIpO1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YW
 luIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ2JzTGFiZWwnLCAoSm9ic1NlcnZpY2UpIC0+XG4gIHRyYW5zY2x1ZGU6IHRydWVcbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6IFwiQFwiXG5cbiAgdGVtcGxhdGU6IFwiPHNwYW4gdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknPjxuZy10cmFuc2NsdWRlPjwvbmctdHJhbnNjbHVkZT48L3NwYW4+XCJcbiAgXG4gIGxpbms
 6IChzY29wZSwgZWxlbWVudCwgYXR0cnMpIC0+XG4gICAgc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IC0+XG4gICAgICAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKVxuXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cblxuLmRpcmVjdGl2ZSAnaW5kaWNhdG9yUHJpbWFyeScsIChKb2JzU2VydmljZSkgLT5cbiAgcmVwbGFjZTogdHJ1ZVxuICBzY29wZTogXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcbiAgICBzdGF0dXM6ICdAJ1xuXG4gIHRlbXBsYXRlOiBcIjxpIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJyAvPlwiXG4gIFxuICBsaW5rOiAoc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSAtPlxuICAgIHNjb3BlLmdldExhYmVsQ2xhc3MgPSAtPlxuICAgICAgJ2ZhIGZhLWNpcmNsZSBpbmRpY2F0b3IgaW5kaWNhdG9yLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cylcblxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXG5cbi5kaXJlY3RpdmUgJ3RhYmxlUHJvcGVydHknLCAtPlxuICByZXBsYWNlOiB0cnVlXG4gIHNjb3BlOlxuICAgIHZhbHVlOiAnPSdcblxuICB0ZW1wbGF0ZTogXCI8dGQgdGl0bGU9XFxcInt7dmFsdWUgfHwgJ05vbmUnfX1cXFwiPnt7dmFsdWUgfHwgJ05vbmUnfX08L3RkPlwiXG4i
 LCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5kaXJlY3RpdmUoJ2JzTGFiZWwnLCBmdW5jdGlvbihKb2JzU2VydmljZSkge1xuICByZXR1cm4ge1xuICAgIHRyYW5zY2x1ZGU6IHRydWUsXG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgZ2V0TGFiZWxDbGFzczogXCImXCIsXG4gICAgICBzdGF0dXM6IFwiQFwiXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8c3BhbiB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKSc+PG5nLXRyYW5zY2x1ZGU+PC9uZy10cmFuc2NsdWRlPjwvc3Bhbj5cIixcbiAgICBsaW5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnbGFiZWwgbGFiZWwtJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCdpbmRpY2F0b3JQcmltYXJ5JywgZnVuY3Rpb24oSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIHtcbiAgICByZXBsYWNlOiB0cnVlLFxuICAgIHNjb3BlOiB7XG4gICAgICBnZXRMYWJlbENsYXNzOiBcIiZcIixcbiAgICAgIHN0YXR1czogJ0AnXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8aSB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKScgLz5cIixcbiAgICBsa
 W5rOiBmdW5jdGlvbihzY29wZSwgZWxlbWVudCwgYXR0cnMpIHtcbiAgICAgIHJldHVybiBzY29wZS5nZXRMYWJlbENsYXNzID0gZnVuY3Rpb24oKSB7XG4gICAgICAgIHJldHVybiAnZmEgZmEtY2lyY2xlIGluZGljYXRvciBpbmRpY2F0b3ItJyArIEpvYnNTZXJ2aWNlLnRyYW5zbGF0ZUxhYmVsU3RhdGUoYXR0cnMuc3RhdHVzKTtcbiAgICAgIH07XG4gICAgfVxuICB9O1xufSkuZGlyZWN0aXZlKCd0YWJsZVByb3BlcnR5JywgZnVuY3Rpb24oKSB7XG4gIHJldHVybiB7XG4gICAgcmVwbGFjZTogdHJ1ZSxcbiAgICBzY29wZToge1xuICAgICAgdmFsdWU6ICc9J1xuICAgIH0sXG4gICAgdGVtcGxhdGU6IFwiPHRkIHRpdGxlPVxcXCJ7e3ZhbHVlIHx8ICdOb25lJ319XFxcIj57e3ZhbHVlIHx8ICdOb25lJ319PC90ZD5cIlxuICB9O1xufSk7XG4iLCIjXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYX
 kgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxuI1xuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXG4jXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyBJU1wiIEJBU0lTLFxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cbiNcblxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcblxuLmZpbHRlciBcImFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZFwiLCAoYW5ndWxhck1vbWVudENvbmZpZykgLT5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gKHZhbHVlLCBmb3JtYXQsIGR1cmF0aW9uRm9ybWF0KSAtPlxuICAgIHJldHVybiBcIlwiICBpZiB0eXBlb2YgdmFsdWUgaXMgXCJ1bmRlZmluZWRcIiBvciB2YWx1ZSBpcyBudWxsXG5cbiAgICBtb21lbnQuZHVyYXRpb24odmFsdWU
 sIGZvcm1hdCkuZm9ybWF0KGR1cmF0aW9uRm9ybWF0LCB7IHRyaW06IGZhbHNlIH0pXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzXG5cbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyXG5cbi5maWx0ZXIgXCJodW1hbml6ZVRleHRcIiwgLT5cbiAgKHRleHQpIC0+XG4gICAgIyBUT0RPOiBleHRlbmQuLi4gYSBsb3RcbiAgICBpZiB0ZXh0IHRoZW4gdGV4dC5yZXBsYWNlKC8mZ3Q7L2csIFwiPlwiKS5yZXBsYWNlKC88YnJcXC8+L2csXCJcIikgZWxzZSAnJ1xuXG4uZmlsdGVyIFwiYnl0ZXNcIiwgLT5cbiAgKGJ5dGVzLCBwcmVjaXNpb24pIC0+XG4gICAgcmV0dXJuIFwiLVwiICBpZiBpc05hTihwYXJzZUZsb2F0KGJ5dGVzKSkgb3Igbm90IGlzRmluaXRlKGJ5dGVzKVxuICAgIHByZWNpc2lvbiA9IDEgIGlmIHR5cGVvZiBwcmVjaXNpb24gaXMgXCJ1bmRlZmluZWRcIlxuICAgIHVuaXRzID0gWyBcImJ5dGVzXCIsIFwia0JcIiwgXCJNQlwiLCBcIkdCXCIsIFwiVEJcIiwgXCJQQlwiIF1cbiAgICBudW1iZXIgPSBNYXRoLmZsb29yKE1hdGgubG9nKGJ5dGVzKSAvIE1hdGgubG9nKDEwMjQpKVxuICAgIChieXRlcyAvIE1hdGgucG93KDEwMjQsIE1hdGguZmxvb3IobnVtYmVyKSkpLnRvRml4ZWQocHJlY2lzaW9uKSArIFwiIFwiICsgdW5pdHNbbnVtYmVyXVxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBw
 JykuZmlsdGVyKFwiYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkXCIsIGZ1bmN0aW9uKGFuZ3VsYXJNb21lbnRDb25maWcpIHtcbiAgdmFyIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyID0gZnVuY3Rpb24odmFsdWUsIGZvcm1hdCwgZHVyYXRpb25Gb3JtYXQpIHtcbiAgICBpZiAodHlwZW9mIHZhbHVlID09PSBcInVuZGVmaW5lZFwiIHx8IHZhbHVlID09PSBudWxsKSB7XG4gICAgICByZXR1cm4gXCJcIjtcbiAgICB9XG4gICAgcmV0dXJuIG1vbWVudC5kdXJhdGlvbih2YWx1ZSwgZm9ybWF0KS5mb3JtYXQoZHVyYXRpb25Gb3JtYXQsIHtcbiAgICAgIHRyaW06IGZhbHNlXG4gICAgfSk7XG4gIH07XG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlci4kc3RhdGVmdWwgPSBhbmd1bGFyTW9tZW50Q29uZmlnLnN0YXRlZnVsRmlsdGVycztcbiAgcmV0dXJuIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlcjtcbn0pLmZpbHRlcihcImh1bWFuaXplVGV4dFwiLCBmdW5jdGlvbigpIHtcbiAgcmV0dXJuIGZ1bmN0aW9uKHRleHQpIHtcbiAgICBpZiAodGV4dCkge1xuICAgICAgcmV0dXJuIHRleHQucmVwbGFjZSgvJmd0Oy9nLCBcIj5cIikucmVwbGFjZSgvPGJyXFwvPi9nLCBcIlwiKTtcbiAgICB9IGVsc2Uge1xuICAgICAgcmV0dXJuICcnO1xuICAgIH1cbiAgfTtcbn0pLmZpbHRlcihcImJ5dGVzXCIsIGZ1bmN0aW9uKCkge1xuICByZXR1c
 m4gZnVuY3Rpb24oYnl0ZXMsIHByZWNpc2lvbikge1xuICAgIHZhciBudW1iZXIsIHVuaXRzO1xuICAgIGlmIChpc05hTihwYXJzZUZsb2F0KGJ5dGVzKSkgfHwgIWlzRmluaXRlKGJ5dGVzKSkge1xuICAgICAgcmV0dXJuIFwiLVwiO1xuICAgIH1cbiAgICBpZiAodHlwZW9mIHByZWNpc2lvbiA9PT0gXCJ1bmRlZmluZWRcIikge1xuICAgICAgcHJlY2lzaW9uID0gMTtcbiAgICB9XG4gICAgdW5pdHMgPSBbXCJieXRlc1wiLCBcImtCXCIsIFwiTUJcIiwgXCJHQlwiLCBcIlRCXCIsIFwiUEJcIl07XG4gICAgbnVtYmVyID0gTWF0aC5mbG9vcihNYXRoLmxvZyhieXRlcykgLyBNYXRoLmxvZygxMDI0KSk7XG4gICAgcmV0dXJuIChieXRlcyAvIE1hdGgucG93KDEwMjQsIE1hdGguZmxvb3IobnVtYmVyKSkpLnRvRml4ZWQocHJlY2lzaW9uKSArIFwiIFwiICsgdW5pdHNbbnVtYmVyXTtcbiAgfTtcbn0pO1xuIiwiI1xuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb2
 4gMi4wICh0aGVcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcbiNcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxuI1xuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXG4jXG5cbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXG5cbi5zZXJ2aWNlICdNYWluU2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRxKSAtPlxuICBAbG9hZENvbmZpZyA9IC0+XG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpXG5cbiAgICAkaHR0cC5nZXQgXCIvY29uZmlnXCJcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25ma

<TRUNCATED>

[22/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
new file mode 100644
index 0000000..1ee89d4
--- /dev/null
+++ b/flink-runtime-web/src/main/resources/web/fonts/fontawesome-webfont.svg
@@ -0,0 +1,565 @@
+<?xml version="1.0" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
+<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1">
+<metadata></metadata>
+<defs>
+<font id="fontawesomeregular" horiz-adv-x="1536" >
+<font-face units-per-em="1792" ascent="1536" descent="-256" />
+<missing-glyph horiz-adv-x="448" />
+<glyph unicode=" "  horiz-adv-x="448" />
+<glyph unicode="&#x09;" horiz-adv-x="448" />
+<glyph unicode="&#xa0;" horiz-adv-x="448" />
+<glyph unicode="&#xa8;" horiz-adv-x="1792" />
+<glyph unicode="&#xa9;" horiz-adv-x="1792" />
+<glyph unicode="&#xae;" horiz-adv-x="1792" />
+<glyph unicode="&#xb4;" horiz-adv-x="1792" />
+<glyph unicode="&#xc6;" horiz-adv-x="1792" />
+<glyph unicode="&#xd8;" horiz-adv-x="1792" />
+<glyph unicode="&#x2000;" horiz-adv-x="768" />
+<glyph unicode="&#x2001;" horiz-adv-x="1537" />
+<glyph unicode="&#x2002;" horiz-adv-x="768" />
+<glyph unicode="&#x2003;" horiz-adv-x="1537" />
+<glyph unicode="&#x2004;" horiz-adv-x="512" />
+<glyph unicode="&#x2005;" horiz-adv-x="384" />
+<glyph unicode="&#x2006;" horiz-adv-x="256" />
+<glyph unicode="&#x2007;" horiz-adv-x="256" />
+<glyph unicode="&#x2008;" horiz-adv-x="192" />
+<glyph unicode="&#x2009;" horiz-adv-x="307" />
+<glyph unicode="&#x200a;" horiz-adv-x="85" />
+<glyph unicode="&#x202f;" horiz-adv-x="307" />
+<glyph unicode="&#x205f;" horiz-adv-x="384" />
+<glyph unicode="&#x2122;" horiz-adv-x="1792" />
+<glyph unicode="&#x221e;" horiz-adv-x="1792" />
+<glyph unicode="&#x2260;" horiz-adv-x="1792" />
+<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
+<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
+<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
+<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
+<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
+<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
+<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
+<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
+<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
+<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
+<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
+<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
+<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
+<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
+<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
+<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
+<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
+<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
+<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
+<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
+<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
+<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
+<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
+<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
+<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
+<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
+<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
+<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
+<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
+<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
+<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
+<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
+<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
+<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
+<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
+<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
+<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
+<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
+<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
+<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
+<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
+<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
+<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
+<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
+<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
+<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
+<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
+<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
+<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
+<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
+<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
+<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
+<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
+<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
+<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
+<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
+<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
+<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
+<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
+<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
+<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
+<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
+<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
+<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
+<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
+<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
+<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
+<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
+<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf082;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 q0 119 84.5 203.5t203.5 84.5h960z" />
+<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
+<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
+<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
+<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
+<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
+<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
+<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
+<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
+<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
+<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
+<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
+<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
+<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
+<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
+<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
+<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
+<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
+<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
+<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
+<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
+<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
+<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
+<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
+<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
+<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
+<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
+<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11

<TRUNCATED>

[42/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/src/main/resources/web/js/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/resources/web/js/index.js b/flink-runtime-web/src/main/resources/web/js/index.js
deleted file mode 100644
index d370765..0000000
--- a/flink-runtime-web/src/main/resources/web/js/index.js
+++ /dev/null
@@ -1,1420 +0,0 @@
-angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", function($rootScope) {
-  $rootScope.sidebarVisible = false;
-  return $rootScope.showSidebar = function() {
-    $rootScope.sidebarVisible = !$rootScope.sidebarVisible;
-    return $rootScope.sidebarClass = 'force-show';
-  };
-}]).value('flinkConfig', {
-  "refresh-interval": 10000
-}).run(["JobsService", "MainService", "flinkConfig", "$interval", function(JobsService, MainService, flinkConfig, $interval) {
-  MainService.loadConfig().then(function(config) {
-    angular.extend(flinkConfig, config);
-    JobsService.listJobs();
-    return $interval(function() {
-      return JobsService.listJobs();
-    }, flinkConfig["refresh-interval"]);
-  });
-  Highcharts.setOptions({
-    global: {
-      useUTC: false
-    }
-  });
-  Highcharts.createElement('link', {
-    href: '//fonts.googleapis.com/css?family=Dosis:400,600',
-    rel: 'stylesheet',
-    type: 'text/css'
-  }, null, document.getElementsByTagName('head')[0]);
-  Highcharts.theme = {
-    colors: ["#7cb5ec", "#f7a35c", "#90ee7e", "#7798BF", "#aaeeee", "#ff0066", "#eeaaee", "#55BF3B", "#DF5353", "#7798BF", "#aaeeee"],
-    chart: {
-      backgroundColor: null,
-      style: {
-        fontFamily: "Dosis, sans-serif"
-      }
-    },
-    title: {
-      style: {
-        fontSize: '16px',
-        fontWeight: 'bold',
-        textTransform: 'uppercase'
-      }
-    },
-    tooltip: {
-      borderWidth: 0,
-      backgroundColor: 'rgba(219,219,216,0.8)',
-      shadow: false
-    },
-    legend: {
-      itemStyle: {
-        fontWeight: 'bold',
-        fontSize: '13px'
-      }
-    },
-    xAxis: {
-      gridLineWidth: 1,
-      labels: {
-        style: {
-          fontSize: '12px'
-        }
-      }
-    },
-    yAxis: {
-      minorTickInterval: 'auto',
-      title: {
-        style: {
-          textTransform: 'uppercase'
-        }
-      },
-      labels: {
-        style: {
-          fontSize: '12px'
-        }
-      }
-    },
-    plotOptions: {
-      candlestick: {
-        lineColor: '#404048'
-      }
-    },
-    background2: '#F0F0EA'
-  };
-  return Highcharts.setOptions(Highcharts.theme);
-}]).config(["$uiViewScrollProvider", function($uiViewScrollProvider) {
-  return $uiViewScrollProvider.useAnchorScroll();
-}]).config(["$stateProvider", "$urlRouterProvider", function($stateProvider, $urlRouterProvider) {
-  $stateProvider.state("overview", {
-    url: "/overview",
-    views: {
-      main: {
-        templateUrl: "partials/overview.html",
-        controller: 'OverviewController'
-      }
-    }
-  }).state("running-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: true,
-    views: {
-      main: {
-        templateUrl: "partials/jobs/job.html",
-        controller: 'SingleJobController'
-      }
-    }
-  }).state("single-job.plan", {
-    url: "",
-    abstract: true,
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.plan.html",
-        controller: 'JobPlanController'
-      }
-    }
-  }).state("single-job.plan.overview", {
-    url: "",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.overview.html",
-        controller: 'JobPlanOverviewController'
-      }
-    }
-  }).state("single-job.plan.accumulators", {
-    url: "/accumulators",
-    views: {
-      'node-details': {
-        templateUrl: "partials/jobs/job.plan.node-list.accumulators.html",
-        controller: 'JobPlanAccumulatorsController'
-      }
-    }
-  }).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.statistics", {
-    url: "/statistics",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.statistics.html"
-      }
-    }
-  }).state("single-job.exceptions", {
-    url: "/exceptions",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.exceptions.html",
-        controller: 'JobExceptionsController'
-      }
-    }
-  }).state("single-job.properties", {
-    url: "/properties",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.properties.html",
-        controller: 'JobPropertiesController'
-      }
-    }
-  }).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}",
-    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.log", {
-    url: "/logfile",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
-      }
-    }
-  }).state("single-manager.stdout", {
-    url: "/stdout",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
-      }
-    }
-  }).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'
-      }
-    }
-  });
-  return $urlRouterProvider.otherwise("/overview");
-}]);
-
-angular.module('flinkApp').directive('bsLabel', ["JobsService", function(JobsService) {
-  return {
-    transclude: true,
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: "@"
-    },
-    template: "<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'label label-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('indicatorPrimary', ["JobsService", function(JobsService) {
-  return {
-    replace: true,
-    scope: {
-      getLabelClass: "&",
-      status: '@'
-    },
-    template: "<i title='{{status}}' ng-class='getLabelClass()' />",
-    link: function(scope, element, attrs) {
-      return scope.getLabelClass = function() {
-        return 'fa fa-circle indicator indicator-' + JobsService.translateLabelState(attrs.status);
-      };
-    }
-  };
-}]).directive('tableProperty', function() {
-  return {
-    replace: true,
-    scope: {
-      value: '='
-    },
-    template: "<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"
-  };
-});
-
-angular.module('flinkApp').filter("amDurationFormatExtended", ["angularMomentConfig", function(angularMomentConfig) {
-  var amDurationFormatExtendedFilter;
-  amDurationFormatExtendedFilter = function(value, format, durationFormat) {
-    if (typeof value === "undefined" || value === null) {
-      return "";
-    }
-    return moment.duration(value, format).format(durationFormat, {
-      trim: false
-    });
-  };
-  amDurationFormatExtendedFilter.$stateful = angularMomentConfig.statefulFilters;
-  return amDurationFormatExtendedFilter;
-}]).filter("humanizeText", function() {
-  return function(text) {
-    if (text) {
-      return text.replace(/&gt;/g, ">").replace(/<br\/>/g, "");
-    } else {
-      return '';
-    }
-  };
-}).filter("bytes", function() {
-  return function(bytes, precision) {
-    var number, units;
-    if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
-      return "-";
-    }
-    if (typeof precision === "undefined") {
-      precision = 1;
-    }
-    units = ["bytes", "kB", "MB", "GB", "TB", "PB"];
-    number = Math.floor(Math.log(bytes) / Math.log(1024));
-    return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + " " + units[number];
-  };
-});
-
-angular.module('flinkApp').service('MainService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("config").success(function(data, status, headers, config) {
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('JobManagerConfigController', ["$scope", "JobManagerConfigService", function($scope, JobManagerConfigService) {
-  return JobManagerConfigService.loadConfig().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['config'] = data;
-  });
-}]).controller('JobManagerLogsController', ["$scope", "JobManagerLogsService", function($scope, JobManagerLogsService) {
-  JobManagerLogsService.loadLogs().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['log'] = data;
-  });
-  return $scope.reloadData = function() {
-    return JobManagerLogsService.loadLogs().then(function(data) {
-      return $scope.jobmanager['log'] = data;
-    });
-  };
-}]).controller('JobManagerStdoutController', ["$scope", "JobManagerStdoutService", function($scope, JobManagerStdoutService) {
-  JobManagerStdoutService.loadStdout().then(function(data) {
-    if ($scope.jobmanager == null) {
-      $scope.jobmanager = {};
-    }
-    return $scope.jobmanager['stdout'] = data;
-  });
-  return $scope.reloadData = function() {
-    return JobManagerStdoutService.loadStdout().then(function(data) {
-      return $scope.jobmanager['stdout'] = data;
-    });
-  };
-}]);
-
-angular.module('flinkApp').service('JobManagerConfigService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var config;
-  config = {};
-  this.loadConfig = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/config").success(function(data, status, headers, config) {
-      config = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('JobManagerLogsService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var logs;
-  logs = {};
-  this.loadLogs = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/log").success(function(data, status, headers, config) {
-      logs = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('JobManagerStdoutService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var stdout;
-  stdout = {};
-  this.loadStdout = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("jobmanager/stdout").success(function(data, status, headers, config) {
-      stdout = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('RunningJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('running');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('CompletedJobsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  $scope.jobObserver = function() {
-    return $scope.jobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  return $scope.jobObserver();
-}]).controller('SingleJobController', ["$scope", "$state", "$stateParams", "JobsService", "$rootScope", "flinkConfig", "$interval", function($scope, $state, $stateParams, JobsService, $rootScope, flinkConfig, $interval) {
-  var refresher;
-  console.log('SingleJobController');
-  $scope.jobid = $stateParams.jobid;
-  $scope.job = null;
-  $scope.plan = null;
-  $scope.vertices = null;
-  JobsService.loadJob($stateParams.jobid).then(function(data) {
-    $scope.job = data;
-    $scope.plan = data.plan;
-    return $scope.vertices = data.vertices;
-  });
-  refresher = $interval(function() {
-    return JobsService.loadJob($stateParams.jobid).then(function(data) {
-      $scope.job = data;
-      return $scope.$broadcast('reload');
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    $scope.job = null;
-    $scope.plan = null;
-    $scope.vertices = null;
-    return $interval.cancel(refresher);
-  });
-}]).controller('JobPlanController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobPlanController');
-  $scope.nodeid = null;
-  $scope.nodeUnfolded = false;
-  $scope.stateList = JobsService.stateList();
-  $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      $scope.accumulators = null;
-      return $scope.$broadcast('reload');
-    } else {
-      $scope.nodeid = null;
-      $scope.nodeUnfolded = false;
-      $scope.vertex = null;
-      $scope.subtasks = null;
-      return $scope.accumulators = null;
-    }
-  };
-  $scope.deactivateNode = function() {
-    $scope.nodeid = null;
-    $scope.nodeUnfolded = false;
-    $scope.vertex = null;
-    $scope.subtasks = null;
-    return $scope.accumulators = null;
-  };
-  return $scope.toggleFold = function() {
-    return $scope.nodeUnfolded = !$scope.nodeUnfolded;
-  };
-}]).controller('JobPlanOverviewController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanOverviewController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.st)) {
-    JobsService.getSubtasks($scope.nodeid).then(function(data) {
-      return $scope.subtasks = data;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanOverviewController');
-    if ($scope.nodeid) {
-      return JobsService.getSubtasks($scope.nodeid).then(function(data) {
-        return $scope.subtasks = data;
-      });
-    }
-  });
-}]).controller('JobPlanAccumulatorsController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPlanAccumulatorsController');
-  if ($scope.nodeid && (!$scope.vertex || !$scope.vertex.accumulators)) {
-    JobsService.getAccumulators($scope.nodeid).then(function(data) {
-      $scope.accumulators = data.main;
-      return $scope.subtaskAccumulators = data.subtasks;
-    });
-  }
-  return $scope.$on('reload', function(event) {
-    console.log('JobPlanAccumulatorsController');
-    if ($scope.nodeid) {
-      return JobsService.getAccumulators($scope.nodeid).then(function(data) {
-        $scope.accumulators = data.main;
-        return $scope.subtaskAccumulators = data.subtasks;
-      });
-    }
-  });
-}]).controller('JobTimelineVertexController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  console.log('JobTimelineVertexController');
-  JobsService.getVertex($stateParams.vertexId).then(function(data) {
-    return $scope.vertex = data;
-  });
-  return $scope.$on('reload', function(event) {
-    console.log('JobTimelineVertexController');
-    return JobsService.getVertex($stateParams.vertexId).then(function(data) {
-      return $scope.vertex = data;
-    });
-  });
-}]).controller('JobExceptionsController', ["$scope", "$state", "$stateParams", "JobsService", function($scope, $state, $stateParams, JobsService) {
-  return JobsService.loadExceptions().then(function(data) {
-    return $scope.exceptions = data;
-  });
-}]).controller('JobPropertiesController', ["$scope", "JobsService", function($scope, JobsService) {
-  console.log('JobPropertiesController');
-  return $scope.changeNode = function(nodeid) {
-    if (nodeid !== $scope.nodeid) {
-      $scope.nodeid = nodeid;
-      return JobsService.getNode(nodeid).then(function(data) {
-        return $scope.node = data;
-      });
-    } else {
-      $scope.nodeid = null;
-      return $scope.node = null;
-    }
-  };
-}]);
-
-angular.module('flinkApp').directive('vertex', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline secondary' width='0' height='0'></svg>",
-    scope: {
-      data: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data.subtasks, function(subtask, i) {
-          var times;
-          times = [
-            {
-              label: "Scheduled",
-              color: "#666",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["SCHEDULED"],
-              ending_time: subtask.timestamps["DEPLOYING"],
-              type: 'regular'
-            }, {
-              label: "Deploying",
-              color: "#aaa",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["DEPLOYING"],
-              ending_time: subtask.timestamps["RUNNING"],
-              type: 'regular'
-            }
-          ];
-          if (subtask.timestamps["FINISHED"] > 0) {
-            times.push({
-              label: "Running",
-              color: "#ddd",
-              borderColor: "#555",
-              starting_time: subtask.timestamps["RUNNING"],
-              ending_time: subtask.timestamps["FINISHED"],
-              type: 'regular'
-            });
-          }
-          return testData.push({
-            label: "(" + subtask.subtask + ") " + subtask.host,
-            times: times
-          });
-        });
-        chart = d3.timeline().stack().tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("single").labelFormat(function(label) {
-          return label;
-        }).margin({
-          left: 100,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).relativeTime();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      analyzeTime(scope.data);
-    }
-  };
-}]).directive('timeline', ["$state", function($state) {
-  return {
-    template: "<svg class='timeline' width='0' height='0'></svg>",
-    scope: {
-      vertices: "=",
-      jobid: "="
-    },
-    link: function(scope, elem, attrs) {
-      var analyzeTime, containerW, svgEl, translateLabel;
-      svgEl = elem.children()[0];
-      containerW = elem.width();
-      angular.element(svgEl).attr('width', containerW);
-      translateLabel = function(label) {
-        return label.replace("&gt;", ">");
-      };
-      analyzeTime = function(data) {
-        var chart, svg, testData;
-        d3.select(svgEl).selectAll("*").remove();
-        testData = [];
-        angular.forEach(data, function(vertex) {
-          if (vertex['start-time'] > -1) {
-            if (vertex.type === 'scheduled') {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#cccccc",
-                    borderColor: "#555555",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    type: vertex.type
-                  }
-                ]
-              });
-            } else {
-              return testData.push({
-                times: [
-                  {
-                    label: translateLabel(vertex.name),
-                    color: "#d9f1f7",
-                    borderColor: "#62cdea",
-                    starting_time: vertex['start-time'],
-                    ending_time: vertex['end-time'],
-                    link: vertex.id,
-                    type: vertex.type
-                  }
-                ]
-              });
-            }
-          }
-        });
-        chart = d3.timeline().stack().click(function(d, i, datum) {
-          if (d.link) {
-            return $state.go("single-job.timeline.vertex", {
-              jobid: scope.jobid,
-              vertexId: d.link
-            });
-          }
-        }).tickFormat({
-          format: d3.time.format("%L"),
-          tickSize: 1
-        }).prefix("main").margin({
-          left: 0,
-          right: 0,
-          top: 0,
-          bottom: 0
-        }).itemHeight(30).showBorderLine().showHourTimeline();
-        return svg = d3.select(svgEl).datum(testData).call(chart);
-      };
-      scope.$watch(attrs.vertices, function(data) {
-        if (data) {
-          return analyzeTime(data);
-        }
-      });
-    }
-  };
-}]).directive('jobPlan', ["$timeout", function($timeout) {
-  return {
-    template: "<svg class='graph' width='500' height='400'><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: '=',
-      setNode: '&'
-    },
-    link: function(scope, elem, attrs) {
-      var containerW, createEdge, createLabelEdge, createLabelNode, createNode, d3mainSvg, d3mainSvgG, d3tmpSvg, drawGraph, extendLabelNodeForIteration, g, getNodeType, isSpecialIterationNode, jobid, loadJsonToDagre, mainG, mainSvgElement, mainTmpElement, mainZoom, searchForNode, shortenString, subgraphs;
-      g = null;
-      mainZoom = d3.behavior.zoom();
-      subgraphs = [];
-      jobid = attrs.jobid;
-      mainSvgElement = elem.children()[0];
-      mainG = elem.children().children()[0];
-      mainTmpElement = elem.children()[1];
-      d3mainSvg = d3.select(mainSvgElement);
-      d3mainSvgG = d3.select(mainG);
-      d3tmpSvg = d3.select(mainTmpElement);
-      containerW = elem.width();
-      angular.element(elem.children()[0]).width(containerW);
-      scope.zoomIn = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() < 2.99) {
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() + 0.1 / (mainZoom.scale()));
-          mainZoom.scale(mainZoom.scale() + 0.1);
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      scope.zoomOut = function() {
-        var translate, v1, v2;
-        if (mainZoom.scale() > 0.31) {
-          mainZoom.scale(mainZoom.scale() - 0.1);
-          translate = mainZoom.translate();
-          v1 = translate[0] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          v2 = translate[1] * (mainZoom.scale() - 0.1 / (mainZoom.scale()));
-          mainZoom.translate([v1, v2]);
-          return d3mainSvgG.attr("transform", "translate(" + v1 + "," + v2 + ") scale(" + mainZoom.scale() + ")");
-        }
-      };
-      createLabelEdge = function(el) {
-        var labelValue;
-        labelValue = "";
-        if ((el.ship_strategy != null) || (el.local_strategy != null)) {
-          labelValue += "<div class='edge-label'>";
-          if (el.ship_strategy != null) {
-            labelValue += el.ship_strategy;
-          }
-          if (el.temp_mode !== undefined) {
-            labelValue += " (" + el.temp_mode + ")";
-          }
-          if (el.local_strategy !== undefined) {
-            labelValue += ",<br>" + el.local_strategy;
-          }
-          labelValue += "</div>";
-        }
-        return labelValue;
-      };
-      isSpecialIterationNode = function(info) {
-        return info === "partialSolution" || info === "nextPartialSolution" || info === "workset" || info === "nextWorkset" || info === "solutionSet" || info === "solutionDelta";
-      };
-      getNodeType = function(el, info) {
-        if (info === "mirror") {
-          return 'node-mirror';
-        } else if (isSpecialIterationNode(info)) {
-          return 'node-iteration';
-        } else {
-          return 'node-normal';
-        }
-      };
-      createLabelNode = function(el, info, maxW, maxH) {
-        var labelValue, stepName;
-        labelValue = "<div href='#/jobs/" + jobid + "/vertex/" + el.id + "' class='node-label " + getNodeType(el, info) + "'>";
-        if (info === "mirror") {
-          labelValue += "<h3 class='node-name'>Mirror of " + el.operator + "</h3>";
-        } else {
-          labelValue += "<h3 class='node-name'>" + el.operator + "</h3>";
-        }
-        if (el.description === "") {
-          labelValue += "";
-        } else {
-          stepName = el.description;
-          stepName = shortenString(stepName);
-          labelValue += "<h4 class='step-name'>" + stepName + "</h4>";
-        }
-        if (el.step_function != null) {
-          labelValue += extendLabelNodeForIteration(el.id, maxW, maxH);
-        } else {
-          if (isSpecialIterationNode(info)) {
-            labelValue += "<h5>" + info + " Node</h5>";
-          }
-          if (el.parallelism !== "") {
-            labelValue += "<h5>Parallelism: " + el.parallelism + "</h5>";
-          }
-          if (el.operator !== undefined) {
-            labelValue += "<h5>Operation: " + shortenString(el.operator_strategy) + "</h5>";
-          }
-        }
-        labelValue += "</div>";
-        return labelValue;
-      };
-      extendLabelNodeForIteration = function(id, maxW, maxH) {
-        var labelValue, svgID;
-        svgID = "svg-" + id;
-        labelValue = "<svg class='" + svgID + "' width=" + maxW + " height=" + maxH + "><g /></svg>";
-        return labelValue;
-      };
-      shortenString = function(s) {
-        var sbr;
-        if (s.charAt(0) === "<") {
-          s = s.replace("<", "&lt;");
-          s = s.replace(">", "&gt;");
-        }
-        sbr = "";
-        while (s.length > 30) {
-          sbr = sbr + s.substring(0, 30) + "<br>";
-          s = s.substring(30, s.length);
-        }
-        sbr = sbr + s;
-        return sbr;
-      };
-      createNode = function(g, data, el, isParent, maxW, maxH) {
-        if (isParent == null) {
-          isParent = false;
-        }
-        if (el.id === data.partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "partialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "partialSolution")
-          });
-        } else if (el.id === data.next_partial_solution) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextPartialSolution", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextPartialSolution")
-          });
-        } else if (el.id === data.workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "workset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "workset")
-          });
-        } else if (el.id === data.next_workset) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "nextWorkset", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "nextWorkset")
-          });
-        } else if (el.id === data.solution_set) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionSet", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionSet")
-          });
-        } else if (el.id === data.solution_delta) {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "solutionDelta", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "solutionDelta")
-          });
-        } else {
-          return g.setNode(el.id, {
-            label: createLabelNode(el, "", maxW, maxH),
-            labelType: 'html',
-            "class": getNodeType(el, "")
-          });
-        }
-      };
-      createEdge = function(g, data, el, existingNodes, pred, missingNodes) {
-        var missingNode;
-        if (existingNodes.indexOf(pred.id) !== -1) {
-          return g.setEdge(pred.id, el.id, {
-            label: createLabelEdge(pred),
-            labelType: 'html',
-            arrowhead: 'normal'
-          });
-        } else {
-          missingNode = searchForNode(data, pred.id);
-          if (!(!missingNode || missingNodes.indexOf(missingNode.id) > -1)) {
-            missingNodes.push(missingNode.id);
-            g.setNode(missingNode.id, {
-              label: createLabelNode(missingNode, "mirror"),
-              labelType: 'html',
-              "class": getNodeType(missingNode, 'mirror')
-            });
-            return g.setEdge(missingNode.id, el.id, {
-              label: createLabelEdge(missingNode),
-              labelType: 'html'
-            });
-          }
-        }
-      };
-      loadJsonToDagre = function(g, data) {
-        var el, existingNodes, isParent, k, l, len, len1, maxH, maxW, missingNodes, pred, r, ref, sg, toIterate;
-        existingNodes = [];
-        missingNodes = [];
-        if (data.nodes != null) {
-          toIterate = data.nodes;
-        } else {
-          toIterate = data.step_function;
-          isParent = true;
-        }
-        for (k = 0, len = toIterate.length; k < len; k++) {
-          el = toIterate[k];
-          maxW = 0;
-          maxH = 0;
-          if (el.step_function) {
-            sg = new dagreD3.graphlib.Graph({
-              multigraph: true,
-              compound: true
-            }).setGraph({
-              nodesep: 20,
-              edgesep: 0,
-              ranksep: 20,
-              rankdir: "LR",
-              marginx: 10,
-              marginy: 10
-            });
-            subgraphs[el.id] = sg;
-            loadJsonToDagre(sg, el);
-            r = new dagreD3.render();
-            d3tmpSvg.select('g').call(r, sg);
-            maxW = sg.graph().width;
-            maxH = sg.graph().height;
-            angular.element(mainTmpElement).empty();
-          }
-          createNode(g, data, el, isParent, maxW, maxH);
-          existingNodes.push(el.id);
-          if (el.inputs != null) {
-            ref = el.inputs;
-            for (l = 0, len1 = ref.length; l < len1; l++) {
-              pred = ref[l];
-              createEdge(g, data, el, existingNodes, pred, missingNodes);
-            }
-          }
-        }
-        return g;
-      };
-      searchForNode = function(data, nodeID) {
-        var el, i, j;
-        for (i in data.nodes) {
-          el = data.nodes[i];
-          if (el.id === nodeID) {
-            return el;
-          }
-          if (el.step_function != null) {
-            for (j in el.step_function) {
-              if (el.step_function[j].id === nodeID) {
-                return el.step_function[j];
-              }
-            }
-          }
-        }
-      };
-      drawGraph = function(data) {
-        var i, newScale, renderer, sg, xCenterOffset, yCenterOffset;
-        g = new dagreD3.graphlib.Graph({
-          multigraph: true,
-          compound: true
-        }).setGraph({
-          nodesep: 70,
-          edgesep: 0,
-          ranksep: 50,
-          rankdir: "LR",
-          marginx: 40,
-          marginy: 40
-        });
-        loadJsonToDagre(g, data);
-        renderer = new dagreD3.render();
-        d3mainSvgG.call(renderer, g);
-        for (i in subgraphs) {
-          sg = subgraphs[i];
-          d3mainSvg.select('svg.svg-' + i + ' g').call(renderer, sg);
-        }
-        newScale = 0.5;
-        xCenterOffset = Math.floor((angular.element(mainSvgElement).width() - g.graph().width * newScale) / 2);
-        yCenterOffset = Math.floor((angular.element(mainSvgElement).height() - g.graph().height * newScale) / 2);
-        mainZoom.scale(newScale).translate([xCenterOffset, yCenterOffset]);
-        d3mainSvgG.attr("transform", "translate(" + xCenterOffset + ", " + yCenterOffset + ") scale(" + mainZoom.scale() + ")");
-        mainZoom.on("zoom", function() {
-          var ev;
-          ev = d3.event;
-          return d3mainSvgG.attr("transform", "translate(" + ev.translate + ") scale(" + ev.scale + ")");
-        });
-        mainZoom(d3mainSvg);
-        return d3mainSvgG.selectAll('.node').on('click', function(d) {
-          return scope.setNode({
-            nodeid: d
-          });
-        });
-      };
-      scope.$watch(attrs.plan, function(newPlan) {
-        if (newPlan) {
-          return drawGraph(newPlan);
-        }
-      });
-    }
-  };
-}]);
-
-angular.module('flinkApp').service('JobsService', ["$http", "flinkConfig", "$log", "amMoment", "$q", "$timeout", function($http, flinkConfig, $log, amMoment, $q, $timeout) {
-  var currentJob, currentPlan, deferreds, jobObservers, jobs, notifyObservers;
-  currentJob = null;
-  currentPlan = null;
-  deferreds = {};
-  jobs = {
-    running: [],
-    finished: [],
-    cancelled: [],
-    failed: []
-  };
-  jobObservers = [];
-  notifyObservers = function() {
-    return angular.forEach(jobObservers, function(callback) {
-      return callback();
-    });
-  };
-  this.registerObserver = function(callback) {
-    return jobObservers.push(callback);
-  };
-  this.unRegisterObserver = function(callback) {
-    var index;
-    index = jobObservers.indexOf(callback);
-    return jobObservers.splice(index, 1);
-  };
-  this.stateList = function() {
-    return ['SCHEDULED', 'DEPLOYING', 'RUNNING', 'FINISHED', 'FAILED', 'CANCELING', 'CANCELED'];
-  };
-  this.translateLabelState = function(state) {
-    switch (state.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(list) {
-    return angular.forEach(list, function(item, jobKey) {
-      if (!(item['end-time'] > -1)) {
-        return item['end-time'] = item['start-time'] + item['duration'];
-      }
-    });
-  };
-  this.processVertices = function(data) {
-    angular.forEach(data.vertices, function(vertex, i) {
-      return vertex.type = 'regular';
-    });
-    return data.vertices.unshift({
-      name: 'Scheduled',
-      'start-time': data.timestamps['CREATED'],
-      'end-time': data.timestamps['CREATED'] + 1,
-      type: 'scheduled'
-    });
-  };
-  this.listJobs = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("joboverview").success((function(_this) {
-      return function(data, status, headers, config) {
-        angular.forEach(data, function(list, listKey) {
-          switch (listKey) {
-            case 'running':
-              return jobs.running = _this.setEndTimes(list);
-            case 'finished':
-              return jobs.finished = _this.setEndTimes(list);
-            case 'cancelled':
-              return jobs.cancelled = _this.setEndTimes(list);
-            case 'failed':
-              return jobs.failed = _this.setEndTimes(list);
-          }
-        });
-        deferred.resolve(jobs);
-        return notifyObservers();
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getJobs = function(type) {
-    return jobs[type];
-  };
-  this.getAllJobs = function() {
-    return jobs;
-  };
-  this.loadJob = function(jobid) {
-    currentJob = null;
-    deferreds.job = $q.defer();
-    $http.get("jobs/" + jobid).success((function(_this) {
-      return function(data, status, headers, config) {
-        _this.setEndTimes(data.vertices);
-        _this.processVertices(data);
-        return $http.get("jobs/" + jobid + "/config").success(function(jobConfig) {
-          data = angular.extend(data, jobConfig);
-          currentJob = data;
-          return deferreds.job.resolve(currentJob);
-        });
-      };
-    })(this));
-    return deferreds.job.promise;
-  };
-  this.getNode = function(nodeid) {
-    var deferred, seekNode;
-    seekNode = function(nodeid, data) {
-      var j, len, node, sub;
-      for (j = 0, len = data.length; j < len; j++) {
-        node = data[j];
-        if (node.id === nodeid) {
-          return node;
-        }
-        if (node.step_function) {
-          sub = seekNode(nodeid, node.step_function);
-        }
-        if (sub) {
-          return sub;
-        }
-      }
-      return null;
-    };
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var foundNode;
-        foundNode = seekNode(nodeid, currentJob.plan.nodes);
-        foundNode.vertex = _this.seekVertex(nodeid);
-        return deferred.resolve(foundNode);
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.seekVertex = function(nodeid) {
-    var j, len, ref, vertex;
-    ref = currentJob.vertices;
-    for (j = 0, len = ref.length; j < len; j++) {
-      vertex = ref[j];
-      if (vertex.id === nodeid) {
-        return vertex;
-      }
-    }
-    return null;
-  };
-  this.getVertex = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        var vertex;
-        vertex = _this.seekVertex(vertexid);
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasktimes").success(function(data) {
-          vertex.subtasks = data.subtasks;
-          return deferred.resolve(vertex);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getSubtasks = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid).success(function(data) {
-          var subtasks;
-          subtasks = data.subtasks;
-          return deferred.resolve(subtasks);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.getAccumulators = function(vertexid) {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/accumulators").success(function(data) {
-          var accumulators;
-          accumulators = data['user-accumulators'];
-          return $http.get("jobs/" + currentJob.jid + "/vertices/" + vertexid + "/subtasks/accumulators").success(function(data) {
-            var subtaskAccumulators;
-            subtaskAccumulators = data.subtasks;
-            return deferred.resolve({
-              main: accumulators,
-              subtasks: subtaskAccumulators
-            });
-          });
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  this.loadExceptions = function() {
-    var deferred;
-    deferred = $q.defer();
-    deferreds.job.promise.then((function(_this) {
-      return function(data) {
-        return $http.get("jobs/" + currentJob.jid + "/exceptions").success(function(exceptions) {
-          currentJob.exceptions = exceptions;
-          return deferred.resolve(exceptions);
-        });
-      };
-    })(this));
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('OverviewController', ["$scope", "OverviewService", "JobsService", "$interval", "flinkConfig", function($scope, OverviewService, JobsService, $interval, flinkConfig) {
-  var refresh;
-  $scope.jobObserver = function() {
-    $scope.runningJobs = JobsService.getJobs('running');
-    return $scope.finishedJobs = JobsService.getJobs('finished');
-  };
-  JobsService.registerObserver($scope.jobObserver);
-  $scope.$on('$destroy', function() {
-    return JobsService.unRegisterObserver($scope.jobObserver);
-  });
-  $scope.jobObserver();
-  OverviewService.loadOverview().then(function(data) {
-    return $scope.overview = data;
-  });
-  refresh = $interval(function() {
-    return OverviewService.loadOverview().then(function(data) {
-      return $scope.overview = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]);
-
-angular.module('flinkApp').service('OverviewService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  var overview;
-  overview = {};
-  this.loadOverview = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("overview").success(function(data, status, headers, config) {
-      overview = data;
-      return deferred.resolve(data);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-angular.module('flinkApp').controller('AllTaskManagersController', ["$scope", "TaskManagersService", "$interval", "flinkConfig", function($scope, TaskManagersService, $interval, flinkConfig) {
-  var refresh;
-  TaskManagersService.loadManagers().then(function(data) {
-    return $scope.managers = data;
-  });
-  refresh = $interval(function() {
-    return TaskManagersService.loadManagers().then(function(data) {
-      return $scope.managers = data;
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]).controller('SingleTaskManagerController', ["$scope", "$stateParams", "SingleTaskManagerService", "$interval", "flinkConfig", function($scope, $stateParams, SingleTaskManagerService, $interval, flinkConfig) {
-  var refresh;
-  $scope.metrics = {};
-  SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
-    return $scope.metrics = data[0];
-  });
-  refresh = $interval(function() {
-    return SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then(function(data) {
-      return $scope.metrics = data[0];
-    });
-  }, flinkConfig["refresh-interval"]);
-  return $scope.$on('$destroy', function() {
-    return $interval.cancel(refresh);
-  });
-}]);
-
-angular.module('flinkApp').directive('livechart', function() {
-  return {
-    link: function(scope, element, attrs) {
-      var getChartOptions, getChartType, getKey1, getKey2, getKey3, getKey4, getYAxisTitle, updateCharts;
-      getChartType = function() {
-        if (attrs.key === "cpuLoad") {
-          return "spline";
-        } else {
-          return "area";
-        }
-      };
-      getYAxisTitle = function() {
-        if (attrs.key === "cpuLoad") {
-          return "CPU Usage(%)";
-        } else {
-          return "Memory(MB)";
-        }
-      };
-      getKey1 = function() {
-        return "memory.total." + attrs.key;
-      };
-      getKey2 = function() {
-        return "memory.heap." + attrs.key;
-      };
-      getKey3 = function() {
-        return "memory.non-heap." + attrs.key;
-      };
-      getKey4 = function() {
-        return "cpuLoad";
-      };
-      getChartOptions = function() {
-        return {
-          title: {
-            text: ' '
-          },
-          chart: {
-            type: getChartType(),
-            zoomType: 'x'
-          },
-          xAxis: {
-            type: 'datetime'
-          },
-          yAxis: {
-            title: {
-              text: getYAxisTitle()
-            },
-            min: attrs.key === "cpuLoad" ? 0 : void 0,
-            max: attrs.key === "cpuLoad" ? 100 : void 0
-          },
-          series: [
-            {
-              name: "Memory: Total",
-              id: getKey1(),
-              data: [],
-              color: "#7cb5ec"
-            }, {
-              name: "Memory: Heap",
-              id: getKey2(),
-              data: [],
-              color: "#434348"
-            }, {
-              name: "Memory: Non-Heap",
-              id: getKey3(),
-              data: [],
-              color: "#90ed7d"
-            }, {
-              name: "CPU Usage",
-              id: getKey4(),
-              data: [],
-              color: "#f7a35c",
-              showInLegend: false
-            }
-          ],
-          legend: {
-            enabled: false
-          },
-          tooltip: {
-            shared: true
-          },
-          exporting: {
-            enabled: false
-          },
-          credits: {
-            enabled: false
-          }
-        };
-      };
-      if (element.highcharts() == null) {
-        element.highcharts(getChartOptions());
-      }
-      scope.$watch(attrs.data, function(value) {
-        return updateCharts(value);
-      });
-      return updateCharts = function(value) {
-        return (function(value) {
-          var chart, divider, heartbeat;
-          heartbeat = value.timeSinceLastHeartbeat;
-          chart = element.highcharts();
-          if (attrs.key === "cpuLoad") {
-            return chart.get(getKey4()).addPoint([heartbeat, +((value.metrics.gauges[getKey4()].value * 100).toFixed(2))], true, false);
-          } else {
-            divider = 1048576;
-            chart.get(getKey1()).addPoint([heartbeat, +((value.metrics.gauges[getKey1()].value / divider).toFixed(2))], true, false);
-            chart.get(getKey2()).addPoint([heartbeat, +((value.metrics.gauges[getKey2()].value / divider).toFixed(2))], true, false);
-            return chart.get(getKey3()).addPoint([heartbeat, +((value.metrics.gauges[getKey3()].value / divider).toFixed(2))], true, false);
-          }
-        })(value);
-      };
-    }
-  };
-});
-
-angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadManagers = function() {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("taskmanagers").success(function(data, status, headers, config) {
-      return deferred.resolve(data['taskmanagers']);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]).service('SingleTaskManagerService', ["$http", "flinkConfig", "$q", function($http, flinkConfig, $q) {
-  this.loadMetrics = function(taskmanagerid) {
-    var deferred;
-    deferred = $q.defer();
-    $http.get("taskmanagers/" + taskmanagerid).success(function(data, status, headers, config) {
-      return deferred.resolve(data['taskmanagers']);
-    });
-    return deferred.promise;
-  };
-  return this;
-}]);
-
-//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5
 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQUNILFlBQVksYUFBYSxLQUFLLFNBQUMsUUFBRDtJQUM1QixRQUFRLE9BQU8sYUFBYTtJQUU1QixZQUFZO0lDNUJaLE9EOEJBLFVBQVUsV0FBQTtNQzdCUixPRDhCQSxZQUFZO09BQ1osWUFBWTs7RUFFaEIsV0FBVyxXQUFXO0lBQ3BCLFFBQVE7TUFDTixRQUFROzs7RUFZWixXQUFXLGNBQWMsUUFBUTtJQUNoQyxNQUFNO0lBQ04sS0FBSztJQUNMLE1BQU07S0FDSixNQUFNLFNBQVMscUJBQXFCLFFBQVE7RUFFL0MsV0FBVyxRQUFRO0lBQ2xCLFFBQVEsQ0FBQyxXQUFXLFdBQVcsV0FB
 VyxXQUFXLFdBQVcsV0FBVyxXQUMxRSxXQUFXLFdBQVcsV0FBVztJQUNsQyxPQUFPO01BQ04saUJBQWlCO01BQ2pCLE9BQU87UUFDTixZQUFZOzs7SUFHZCxPQUFPO01BQ04sT0FBTztRQUNOLFVBQVU7UUFDVixZQUFZO1FBQ1osZUFBZTs7O0lBR2pCLFNBQVM7TUFDUixhQUFhO01BQ2IsaUJBQWlCO01BQ2pCLFFBQVE7O0lBRVQsUUFBUTtNQUNQLFdBQVc7UUFDVixZQUFZO1FBQ1osVUFBVTs7O0lBR1osT0FBTztNQUNOLGVBQWU7TUFDZixRQUFRO1FBQ1AsT0FBTztVQUNOLFVBQVU7Ozs7SUFJYixPQUFPO01BQ04sbUJBQW1CO01BQ25CLE9BQU87UUFDTixPQUFPO1VBQ04sZUFBZTs7O01BR2pCLFFBQVE7UUFDUCxPQUFPO1VBQ04sVUFBVTs7OztJQUliLGFBQWE7TUFDWixhQUFhO1FBQ1osV0FBVzs7O0lBSWIsYUFBYTs7RUN4Q2QsT0Q0Q0EsV0FBVyxXQUFXLFdBQVc7SUFLbEMsaUNBQU8sU0FBQyx1QkFBRDtFQy9DTixPRGdEQSxzQkFBc0I7SUFJdkIsZ0RBQU8sU0FBQyxnQkFBZ0Isb0JBQWpCO0VBQ04sZUFBZSxNQUFNLFlBQ25CO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGNBQ0w7SUFBQSxLQUFLO0lBQ
 0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLG1CQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSw0QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0NBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHVCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sOEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFFBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0g7SUFBQSxLQUFLO0
 lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRW5CLE1BQU0sMEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sY0FDSDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7OztLQUVwQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7Ozs7RUMxQmxCLE9ENEJBLG1CQUFtQixVQUFVOztBQzFCL0I7QUNoUEEsUUFBUSxPQUFPLFlBSWQsVUFBVSwyQkFBVyxTQUFDLGFBQUQ7RUNyQnBCLE9Ec0JBO0lBQUEsWUFBWTtJQUNaLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsaUJBQWlCLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJNUQsVUFBVSxvQ0F
 Bb0IsU0FBQyxhQUFEO0VDckI3QixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsc0NBQXNDLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJakYsVUFBVSxpQkFBaUIsV0FBQTtFQ3JCMUIsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLE9BQU87O0lBRVQsVUFBVTs7O0FDbEJaO0FDcEJBLFFBQVEsT0FBTyxZQUVkLE9BQU8sb0RBQTRCLFNBQUMscUJBQUQ7RUFDbEMsSUFBQTtFQUFBLGlDQUFpQyxTQUFDLE9BQU8sUUFBUSxnQkFBaEI7SUFDL0IsSUFBYyxPQUFPLFVBQVMsZUFBZSxVQUFTLE1BQXREO01BQUEsT0FBTzs7SUNoQlAsT0RrQkEsT0FBTyxTQUFTLE9BQU8sUUFBUSxPQUFPLGdCQUFnQjtNQUFFLE1BQU07OztFQUVoRSwrQkFBK0IsWUFBWSxvQkFBb0I7RUNmL0QsT0RpQkE7SUFFRCxPQUFPLGdCQUFnQixXQUFBO0VDakJ0QixPRGtCQSxTQUFDLE1BQUQ7SUFFRSxJQUFHLE1BQUg7TUNsQkUsT0RrQlcsS0FBSyxRQUFRLFNBQVMsS0FBSyxRQUFRLFdBQVU7V0FBMUQ7TUNoQkUsT0RnQmlFOzs7R0FFdEUsT0FBTyxTQUFTLFdBQUE7RUNkZixPRGVBLFNBQUMsT0FBTyxXQUFSO0lBQ0UsSUFBQSxRQUFBO0lBQUEsSUFBZSxNQUFNLFdBQVcsV0FBVyxDQUFJLFNBQVMsUUFBeEQ7TUFBQSxPQUFPOztJQUNQLElBQWtCLE9BQU8sY0FB
 YSxhQUF0QztNQUFBLFlBQVk7O0lBQ1osUUFBUSxDQUFFLFNBQVMsTUFBTSxNQUFNLE1BQU0sTUFBTTtJQUMzQyxTQUFTLEtBQUssTUFBTSxLQUFLLElBQUksU0FBUyxLQUFLLElBQUk7SUNUL0MsT0RVQSxDQUFDLFFBQVEsS0FBSyxJQUFJLE1BQU0sS0FBSyxNQUFNLFVBQVUsUUFBUSxhQUFhLE1BQU0sTUFBTTs7O0FDUGxGO0FDaEJBLFFBQVEsT0FBTyxZQUVkLFFBQVEsOENBQWUsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDdEIsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFVBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9Ec0JBOztBQ3BCRjtBQ09BLFFBQVEsT0FBTyxZQUVkLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQ25CeEMsT0RvQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7SUFDeEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNsQnRCLE9EbUJBLE9BQU8sV0FBVyxZQUFZOztJQUVqQyxXQUFXLGdFQUE0QixTQUFDLFFBQVEsdUJBQVQ7RUFDdEMsc0JBQXNCLFdBQVcsS0FBSyxTQUFDLE1BQUQ7SUFDcEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNqQnRCLE9Ea0JBLE9BQU8sV0FBVyxTQUFTOztFQ2hCN0IsT0RrQkEsT0FBTyxhQUFhLFdBQUE7SUNqQmxCLE9Ea0JBLHNCQUFzQixXQUFXLEtBQ
 UssU0FBQyxNQUFEO01DakJwQyxPRGtCQSxPQUFPLFdBQVcsU0FBUzs7O0lBRWhDLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQUN4Qyx3QkFBd0IsYUFBYSxLQUFLLFNBQUMsTUFBRDtJQUN4QyxJQUFJLE9BQUEsY0FBQSxNQUFKO01BQ0UsT0FBTyxhQUFhOztJQ2Z0QixPRGdCQSxPQUFPLFdBQVcsWUFBWTs7RUNkaEMsT0RnQkEsT0FBTyxhQUFhLFdBQUE7SUNmbEIsT0RnQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7TUNmeEMsT0RnQkEsT0FBTyxXQUFXLFlBQVk7Ozs7QUNacEM7QUNkQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDBEQUEyQixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNsQyxJQUFBO0VBQUEsU0FBUztFQUVULEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxxQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxTQUFTO01DcEJULE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9EcUJBO0lBRUQsUUFBUSx3REFBeUIsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDaEMsSUFBQTtFQUFBLE9BQU87RUFFUCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksa0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsT0FBTztNQ3RCUCxPRHVCQSxTQUFTLFFBQVE7O0lDckJuQixPRHVCQSxTQUFTOztFQ3JCWCxPRHVCQTtJQUVELFFBQVEsME
 RBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHFCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUN4QlQsT0R5QkEsU0FBUyxRQUFROztJQ3ZCbkIsT0R5QkEsU0FBUzs7RUN2QlgsT0R5QkE7O0FDdkJGO0FDdEJBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyxxSEFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQVksYUFBYSxXQUFyRTtFQUNqQyxJQUFBO0VBQUEsUUFBUSxJQUFJO0VBRVosT0FBTyxRQUFRLGFBQWE7RUFDNUIsT0FBTyxNQUFNO0VBQ2I
 sT0FBTyxPQUFPO0VBQ2QsT0FBTyxXQUFXO0VBRWxCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUFDM0MsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPLEtBQUs7SUMxQm5CLE9EMkJBLE9BQU8sV0FBVyxLQUFLOztFQUV6QixZQUFZLFVBQVUsV0FBQTtJQzFCcEIsT0QyQkEsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtNQUMzQyxPQUFPLE1BQU07TUMxQmIsT0Q0QkEsT0FBTyxXQUFXOztLQUVwQixZQUFZO0VDM0JkLE9ENkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUFDckIsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPO0lBQ2QsT0FBTyxXQUFXO0lDNUJsQixPRDhCQSxVQUFVLE9BQU87O0lBS3BCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDL0IsUUFBUSxJQUFJO0VBRVosT0FBTyxTQUFTO0VBQ2hCLE9BQU8sZUFBZTtFQUN0QixPQUFPLFlBQVksWUFBWTtFQUUvQixPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01BQ2hCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUFDbEIsT0FBTyxlQUFlO01DbEN0QixPRG9DQSxPQUFPLFdBQVc7V0FOcEI7TUFTRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxlQUFlO01BQ3RCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUNwQ2xCLE9EcUNBLE9BQU8sZUFBZTs7O0VBRTFCLE9BQU8saUJBQWlCLFdBQUE7SUFDdEIsT0FBTyxTQUFTO0lBQ2hCLE9BQU8s
 ZUFBZTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxXQUFXO0lDbkNsQixPRG9DQSxPQUFPLGVBQWU7O0VDbEN4QixPRG9DQSxPQUFPLGFBQWEsV0FBQTtJQ25DbEIsT0RvQ0EsT0FBTyxlQUFlLENBQUMsT0FBTzs7SUFJakMsV0FBVyx1REFBNkIsU0FBQyxRQUFRLGFBQVQ7RUFDdkMsUUFBUSxJQUFJO0VBRVosSUFBRyxPQUFPLFdBQVksQ0FBQyxPQUFPLFVBQVUsQ0FBQyxPQUFPLE9BQU8sS0FBdkQ7SUFDRSxZQUFZLFlBQVksT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01DdEMxQyxPRHVDQSxPQUFPLFdBQVc7OztFQ3BDdEIsT0RzQ0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQUNaLElBQUcsT0FBTyxRQUFWO01DckNFLE9Ec0NBLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUNyQzFDLE9Ec0NBLE9BQU8sV0FBVzs7OztJQUl6QixXQUFXLDJEQUFpQyxTQUFDLFFBQVEsYUFBVDtFQUMzQyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxlQUF2RDtJQUNFLFlBQVksZ0JBQWdCLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtNQUM5QyxPQUFPLGVBQWUsS0FBSztNQ3RDM0IsT0R1Q0EsT0FBTyxzQkFBc0IsS0FBSzs7O0VDcEN0QyxPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRL
 EtBQUssU0FBQyxNQUFEO1FBQzlDLE9BQU8sZUFBZSxLQUFLO1FDckMzQixPRHNDQSxPQUFPLHNCQUFzQixLQUFLOzs7O0lBSXpDLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDekMsUUFBUSxJQUFJO0VBRVosWUFBWSxVQUFVLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ3RDaEQsT0R1Q0EsT0FBTyxTQUFTOztFQ3JDbEIsT0R1Q0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQ3RDWixPRHVDQSxZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO01DdENoRCxPRHVDQSxPQUFPLFNBQVM7OztJQUlyQixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDdkNyQyxPRHdDQSxZQUFZLGlCQUFpQixLQUFLLFNBQUMsTUFBRDtJQ3ZDaEMsT0R3Q0EsT0FBTyxhQUFhOztJQUl2QixXQUFXLHFEQUEyQixTQUFDLFFBQVEsYUFBVDtFQUNyQyxRQUFRLElBQUk7RUN6Q1osT0QyQ0EsT0FBTyxhQUFhLFNBQUMsUUFBRDtJQUNsQixJQUFHLFdBQVUsT0FBTyxRQUFwQjtNQUNFLE9BQU8sU0FBUztNQzFDaEIsT0Q0Q0EsWUFBWSxRQUFRLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUMzQy9CLE9ENENBLE9BQU8sT0FBTzs7V0FKbEI7TUFPRSxPQUFPLFNBQVM7TUMzQ2hCLE9ENENBLE9BQU8sT0FBTzs7OztBQ3hDcEI7QUNuSEEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TU
 FBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBO01BQUEsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUztNQUVyQyxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsT0FBQSxLQUFBO1FBQUEsR0FBRyxPQUFPLE9BQU8sVUFBVSxLQUFLO1FBRWhDLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsU0FBUyxHQUFWO1VBQzdCLElBQUE7VUFBQSxRQUFRO1lBQ047Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNO2VBRVI7Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUFJVixJQUFHLFFBQVEsV0FBVyxjQUFjLEdBQXBDO1lBQ0UsTUFBTSxLQUFLO2NBQ1QsT0FBTztjQUNQLE9BQU87Y0FDUCxhQUFhO2NBQ2IsZUFBZSxRQUFRLFdBQVc7Y0FDbEMsYUFBYSxRQUFRLFdBQVc7Y0FDaEMsTUFBTTs7O1VDdEJSLE9EeUJGLFNBQVMsS0FBSztZQUNaLE9BQU8sTUFBSSxRQUFRLFVBQVEsT0FBSSxRQUFRO1lBQ3ZDLE9BQU87OztRQUdYLFFBQVEsR0FBRyxXQUFXLFFBQ3JCLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLFVBQVU7V0FFWCxPQUFPLFVBQ1AsWUFBWSxTQUFDLE9BQUQ7VUM1QlQ
 sT0Q2QkY7V0FFRCxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTtXQUM5QyxXQUFXLElBQ1g7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQ2hDckIsT0RpQ0E7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLFVBQVU7TUFDVixPQUFPOztJQUVULE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGlCQUFpQixTQUFDLE9BQUQ7UUNqQ2IsT0RrQ0YsTUFBTSxRQUFRLFFBQVE7O01BRXhCLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxNQUFNLFNBQUMsUUFBRDtVQUNwQixJQUFHLE9BQU8sZ0JBQWdCLENBQUMsR0FBM0I7WUFDRSxJQUFHLE9BQU8sU0FBUSxhQUFsQjtjQ2xDSSxPRG1DRixTQUFTLEtBQ1A7Z0JBQUEsT0FBTztrQkFDTDtvQkFBQSxPQUFPLGVBQWUsT0FBTztvQkFDN0IsT0FBTztvQkFDUCxhQUFhO29CQUNiLGVBQWUsT0FBTztvQkFDdEIsYUFBYSxPQUFPO29CQUNwQixNQUFNLE9BQU87Ozs7bUJBUm5CO2NDckJJLE9EZ0NGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29C
 QUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTztvQkFDYixNQUFNLE9BQU87Ozs7Ozs7UUFHdkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUFDbEMsSUFBRyxFQUFFLE1BQUw7WUMxQkksT0QyQkYsT0FBTyxHQUFHLDhCQUE4QjtjQUFFLE9BQU8sTUFBTTtjQUFPLFVBQVUsRUFBRTs7O1dBRzdFLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBR3ZCLFVBQVU7V0FFWCxPQUFPLFFBQ1AsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7V0FDNUMsV0FBVyxJQUNYLGlCQUNBO1FDMUJDLE9ENEJGLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUs7O01BRVIsTUFBTSxPQUFPLE1BQU0sVUFBVSxTQUFDLE1BQUQ7UUFDM0IsSUFBcUIsTUFBckI7VUM3QkksT0Q2QkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUM3QnBCLE9EOEJBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNO01BQ04sU0FBUzs7SUFFWCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsR0FBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLElBQUk7TUFDSixXQ
 UFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFLckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDMUN2QixPRDZDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM1Q3ZCLE9EK0NGLFdBQVcsS0FBSyxhQU
 FhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3RDZCxPRHVDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDeENyQixPRHlDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDeENJLE9EeUNGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUN6Q0QsT0QwQ0Y7ZUFERztVQ3ZDRCxPRDJDQTs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFFaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSx1QkFBdUIsUUFBUSxhQUFhLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHbEgsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLFdBQVc7ZUFEbkU7VUFHRSxjQUFjLDJCQUEyQixHQUFHLFdBQVc7O1FBQ3pELElBQUcsR0FBRyxnQkFBZSxJQUF
 yQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUF3RixHQUFHLGFBQVksV0FBdkc7WUFBQSxjQUFjLG9CQUFvQixjQUFjLEdBQUcscUJBQXFCOzs7UUFHMUUsY0FBYztRQzNDWixPRDRDRjs7TUFHRiw4QkFBOEIsU0FBQyxJQUFJLE1BQU0sTUFBWDtRQUM1QixJQUFBLFlBQUE7UUFBQSxRQUFRLFNBQVM7UUFFakIsYUFBYSxpQkFBaUIsUUFBUSxhQUFhLE9BQU8sYUFBYSxPQUFPO1FDNUM1RSxPRDZDRjs7TUFHRixnQkFBZ0IsU0FBQyxHQUFEO1FBRWQsSUFBQTtRQUFBLElBQUcsRUFBRSxPQUFPLE9BQU0sS0FBbEI7VUFDRSxJQUFJLEVBQUUsUUFBUSxLQUFLO1VBQ25CLElBQUksRUFBRSxRQUFRLEtBQUs7O1FBQ3JCLE1BQU07UUFDTixPQUFNLEVBQUUsU0FBUyxJQUFqQjtVQUNFLE1BQU0sTUFBTSxFQUFFLFVBQVUsR0FBRyxNQUFNO1VBQ2pDLElBQUksRUFBRSxVQUFVLElBQUksRUFBRTs7UUFDeEIsTUFBTSxNQUFNO1FDM0NWLE9ENENGOztNQUVGLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxVQUFrQixNQUFNLE1BQXRDO1FDM0NU
 LElBQUksWUFBWSxNQUFNO1VEMkNDLFdBQVc7O1FBRXBDLElBQUcsR0FBRyxPQUFNLEtBQUssa0JBQWpCO1VDekNJLE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLG1CQUFtQixNQUFNO1lBQ3BELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyx1QkFBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksdUJBQXVCLE1BQU07WUFDeEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLFNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLFdBQVcsTUFBTTtZQUM1QyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGdCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBT
 yxnQkFBZ0IsSUFBSSxpQkFBaUIsTUFBTTtZQUNsRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBSnRCO1VDbkNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLElBQUksTUFBTTtZQUNyQyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7Ozs7TUFFN0IsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTSxjQUFuQztRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3RDSSxPRHVDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFFdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxhQUFhLFFBQVEsWUFBWSxNQUFNLENBQUMsSUFBL0Q7WUFDRSxhQUFhLEtBQUssWUFBWTtZQUM5QixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ3RDaEMsT0R3Q0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsY0FBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBQ2hCLGVBQWU7UUFFZixJQUFHLE
 tBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUN2Q0ksS0FBSyxVQUFVO1VEd0NqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLFVBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDMUNJLE9BQU8sSUFBSTtjRDJDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTTs7OztRQ3RDakQsT0R3Q0Y7O01BR0YsZ0JBQWdCLFNBQUMsTUFBTSxRQUFQO1FBQ2QsSUFBQSxJQUFBLEdBQUE7UUFBQSxLQUFBLEtBQUEsS0FBQSxPQUFBO1VBQ0UsS0FBSyx
 LQUFLLE1BQU07VUFDaEIsSUFBYyxHQUFHLE9BQU0sUUFBdkI7WUFBQSxPQUFPOztVQUdQLElBQUcsR0FBQSxpQkFBQSxNQUFIO1lBQ0UsS0FBQSxLQUFBLEdBQUEsZUFBQTtjQUNFLElBQStCLEdBQUcsY0FBYyxHQUFHLE9BQU0sUUFBekQ7Z0JBQUEsT0FBTyxHQUFHLGNBQWM7Ozs7OztNQUVoQyxZQUFZLFNBQUMsTUFBRDtRQUNWLElBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQ2pDSSxLQUFLLFVBQVU7VURrQ2pCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdB
 QUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ25DTixPRG9DRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUFFckYsU0FBUztRQ25DUCxPRHFDRixXQUFXLFVBQVUsU0FBUyxHQUFHLFNBQVMsU0FBQyxHQUFEO1VDcEN0QyxPRHFDRixNQUFNLFFBQVE7WUFBRSxRQUFROzs7O01BRTVCLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDakNJLE9EaUNKLFVBQVU7Ozs7OztBQzNCaEI7QUMxYUEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBRWQsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3JCaEIsT0RzQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DckI1QixPRHNCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDcEJsQixPRHFCQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNuQjdCLE9Eb0JBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ25CWCxPRG9CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc
 0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDNUJILE9ENEJtQjtNQUR2QixLQUVPO1FDM0JILE9EMkJpQjtNQUZyQixLQUdPO1FDMUJILE9EMEJvQjtNQUh4QixLQUlPO1FDekJILE9EeUJvQjtNQUp4QixLQUtPO1FDeEJILE9Ed0JrQjtNQUx0QixLQU1PO1FDdkJILE9EdUJvQjtNQU54QixLQU9PO1FDdEJILE9Ec0JrQjtNQVB0QixLQVFPO1FDckJILE9EcUJnQjtNQVJwQjtRQ1hJLE9Eb0JHOzs7RUFFVCxLQUFDLGNBQWMsU0FBQyxNQUFEO0lDbEJiLE9EbUJBLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxRQUFQO01BQ3BCLElBQUEsRUFBTyxLQUFLLGNBQWMsQ0FBQyxJQUEzQjtRQ2xCRSxPRG1CQSxLQUFLLGNBQWMsS0FBSyxnQkFBZ0IsS0FBSzs7OztFQUVuRCxLQUFDLGtCQUFrQixTQUFDLE1BQUQ7SUFDakIsUUFBUSxRQUFRLEtBQUssVUFBVSxTQUFDLFFBQVEsR0FBVDtNQ2hCN0IsT0RpQkEsT0FBTyxPQUFPOztJQ2ZoQixPRGlCQSxLQUFLLFNBQVMsUUFBUTtNQUNwQixNQUFNO01BQ04sY0FBYyxLQUFLLFdBQVc7TUFDOUIsWUFBWSxLQUFLLFdBQVcsYUFBYTtNQUN6QyxNQUFNOzs7RUFHVixLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZUFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1
 lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxVQUFVLE9BQ25CLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFVBQVUsUUFBUSxXQUMzQixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE
 7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQyxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQzlELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RF
 RixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsVUFDbkQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDOUQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDOUQsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ES
 nlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUNwQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFlBQ1
 QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLDJGQUE2QixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDdkMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDakJkLE9EbUJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNsQnJCLE9EbUJBLFVBQVUsT0FBTzs7SUFFcEIsV0FBVyxrSEFBK0IsU0FBQyxRQUFRLGNBQWMsMEJBQTBCLFdBQVcsYUFBNUQ7RUFDekMsSUFBQTtFQUFBLE9BQU8sVUFBVTtFQUNqQix5QkFBeUIsWUFBWSxhQUFhLGVBQWUsS0FBSyxTQUFDLE1BQUQ7SUNqQnBFLE9Ea0JFLE9BQU8sVUFBVSxLQUFLOztFQUV4QixVQUFVLFVBQVUsV0FBQTtJQ2pCcEIsT0RrQkUseUJBQXlCLFlBQVksYUFBYSxlQUFlLEtBQUssU0FBQyxNQUFEO01DakJ0RSxPRGtCRSxPQUFPLFVBQVUsS0FBSzs7S0FDeEIsWUFBWTtFQ2hCaEIsT0RrQkUsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2pCdkIsT0RrQkUsVUFBVSxPQUFPOzs7QUNmdkI7QUNWQSxRQUFRLE9BQU8sWUFFZCxVQUFVLGFBQWEsV0FBQTtFQ25CdEI
 sT0RvQkE7SUFDRSxNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01BQ0osSUFBQSxpQkFBQSxjQUFBLFNBQUEsU0FBQSxTQUFBLFNBQUEsZUFBQTtNQUFBLGVBQWUsV0FBQTtRQUNiLElBQUcsTUFBTSxRQUFPLFdBQWhCO1VDbEJFLE9EbUJBO2VBREY7VUNoQkUsT0RtQkE7OztNQUVKLGdCQUFnQixXQUFBO1FBQ2QsSUFBRyxNQUFNLFFBQU8sV0FBaEI7VUNqQkUsT0RrQkE7ZUFERjtVQ2ZFLE9Ea0JBOzs7TUFFSixVQUFVLFdBQUE7UUNoQlIsT0RpQkEsa0JBQWtCLE1BQU07O01BQzFCLFVBQVUsV0FBQTtRQ2ZSLE9EZ0JBLGlCQUFpQixNQUFNOztNQUN6QixVQUFVLFdBQUE7UUNkUixPRGVBLHFCQUFxQixNQUFNOztNQUM3QixVQUFVLFdBQUE7UUNiUixPRGNBOztNQUVGLGtCQUFrQixXQUFBO1FDYmhCLE9EYXNCO1VBQ3RCLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFBQyxNQUFNO1lBQWdCLFVBQVU7O1VBQ3hDLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFDTCxPQUFPO2NBQUMsTUFBTTs7WUFDZCxLQUFVLE1BQU0sUUFBTyxZQUFsQixJQUFBLEtBQUE7WUFDTCxLQUFZLE1BQU0sUUFBTyxZQUFwQixNQUFBLEtBQUE7O1VBRVAsUUFBUTtZQUNOO2NBQUMsTUFBTTtjQUFpQixJQUFJO2NBQVcsTUFBTTtjQUFJLE9BQU87ZUFDeEQ7Y0FBQyxNQUFNO2NBQWdCLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztlQUN2RDtjQUFDLE1BQU07Y0FBb0IsSUFBSTtjQUFXLE1BQU07Y0FBSSxPQUFPO2VBQzNEO2NBQUMsTUFB
 TTtjQUFhLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztjQUFXLGNBQWM7OztVQUUvRSxRQUFRO1lBQUMsU0FBUzs7VUFDbEIsU0FBUztZQUFDLFFBQVE7O1VBQ2xCLFdBQVc7WUFBQyxTQUFTOztVQUNyQixTQUFTO1lBQUMsU0FBUzs7OztNQUdyQixJQUFJLFFBQUEsZ0JBQUEsTUFBSjtRQUNFLFFBQVEsV0FBVzs7TUFFckIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLE9BQUQ7UUN1QnZCLE9EdEJBLGFBQWE7O01Dd0JmLE9EckJBLGVBQWUsU0FBQyxPQUFEO1FDc0JiLE9EckJFLENBQUEsU0FBQyxPQUFEO1VBQ0EsSUFBQSxPQUFBLFNBQUE7VUFBQSxZQUFZLE1BQU07VUFDbEIsUUFBUSxRQUFRO1VBQ2hCLElBQUcsTUFBTSxRQUFPLFdBQWhCO1lDdUJFLE9EdEJBLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLEtBQUssUUFBUSxNQUNsRSxNQUFNO2lCQUhYO1lBS0UsVUFBVTtZQUNWLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLFNBQVMsUUFBUSxNQUN0RSxNQUFNO1lBQ1QsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQUFRLE1BQ3RFLE1BQU07WUNpQlQsT0RoQkEsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQ
 UFRLE1BQ3RFLE1BQU07O1dBakJWOzs7OztBQ3NDWDtBQzdGQSxRQUFRLE9BQU8sWUFFZCxRQUFRLHNEQUF1QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUM5QixLQUFDLGVBQWUsV0FBQTtJQUNkLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZ0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUSxLQUFLOztJQ25CeEIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7SUFFRCxRQUFRLDJEQUE0QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNuQyxLQUFDLGNBQWMsU0FBQyxlQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxrQkFBa0IsZUFDM0IsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DdEJQLE9EdUJBLFNBQVMsUUFBUSxLQUFLOztJQ3JCeEIsT0R1QkEsU0FBUzs7RUNyQlgsT0R1QkE7O0FDckJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbG
 VcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5ydW4gKCRyb290U2NvcGUpIC0+XHJcbiAgJHJvb3RTY29wZS5zaWR
 lYmFyVmlzaWJsZSA9IGZhbHNlXHJcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XHJcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcclxuICAgICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4udmFsdWUgJ2ZsaW5rQ29uZmlnJywge1xyXG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxyXG59XHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4ucnVuIChKb2JzU2VydmljZSwgTWFpblNlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIC0+XHJcbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cclxuICAgIGFuZ3VsYXIuZXh0ZW5kIGZsaW5rQ29uZmlnLCBjb25maWdcclxuXHJcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXHJcblxyXG4gICAgJGludGVydmFsIC0+XHJcbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcclxuICAgICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gIEhpZ2hjaGFydHMuc2V0T3B0aW9ucyh7XHJcbiAgICBnbG9iYWw6IHtcclxuICAgICAgdXNlVVRDOiBmYWxzZVxyXG4gICAgfVxyXG4gIH0pXHJcblxyXG4gICNcclxuICAjIEdyaWQtbGlnaHQgdGhlbWUgZm9yIEhpZ2hj
 aGFydHMgSlNcclxuICAjIEBhdXRob3IgVG9yc3RlaW4gSG9uc2lcclxuICAjXHJcbiAgIyBUYWtlbiBmcm9tIGh0dHBzOi8vZ2l0aHViLmNvbS9oaWdoc2xpZGUtc29mdHdhcmUvaGlnaGNoYXJ0cy5jb21cclxuICAjXHJcblxyXG5cclxuICBIaWdoY2hhcnRzLmNyZWF0ZUVsZW1lbnQoJ2xpbmsnLCB7XHJcbiAgXHRocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxyXG4gIFx0cmVsOiAnc3R5bGVzaGVldCcsXHJcbiAgXHR0eXBlOiAndGV4dC9jc3MnXHJcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XHJcblxyXG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XHJcbiAgXHRjb2xvcnM6IFtcIiM3Y2I1ZWNcIiwgXCIjZjdhMzVjXCIsIFwiIzkwZWU3ZVwiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCIsIFwiI2ZmMDA2NlwiLCBcIiNlZWFhZWVcIixcclxuICBcdFx0XCIjNTVCRjNCXCIsIFwiI0RGNTM1M1wiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCJdLFxyXG4gIFx0Y2hhcnQ6IHtcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiBudWxsLFxyXG4gIFx0XHRzdHlsZToge1xyXG4gIFx0XHRcdGZvbnRGYW1pbHk6IFwiRG9zaXMsIHNhbnMtc2VyaWZcIlxyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dGl0bGU6IHtcclxuICBcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRmb250U2l6ZTogJzE2cHgnLFxyXG4gIFx0XHRcd
 GZvbnRXZWlnaHQ6ICdib2xkJyxcclxuICBcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dG9vbHRpcDoge1xyXG4gIFx0XHRib3JkZXJXaWR0aDogMCxcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiAncmdiYSgyMTksMjE5LDIxNiwwLjgpJyxcclxuICBcdFx0c2hhZG93OiBmYWxzZVxyXG4gIFx0fSxcclxuICBcdGxlZ2VuZDoge1xyXG4gIFx0XHRpdGVtU3R5bGU6IHtcclxuICBcdFx0XHRmb250V2VpZ2h0OiAnYm9sZCcsXHJcbiAgXHRcdFx0Zm9udFNpemU6ICcxM3B4J1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0eEF4aXM6IHtcclxuICBcdFx0Z3JpZExpbmVXaWR0aDogMSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHR5QXhpczoge1xyXG4gIFx0XHRtaW5vclRpY2tJbnRlcnZhbDogJ2F1dG8nLFxyXG4gIFx0XHR0aXRsZToge1xyXG4gIFx0XHRcdHN0eWxlOiB7XHJcbiAgXHRcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHRcdH1cclxuICBcdFx0fSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHRwbG90T3B0aW9ucz
 oge1xyXG4gIFx0XHRjYW5kbGVzdGljazoge1xyXG4gIFx0XHRcdGxpbmVDb2xvcjogJyM0MDQwNDgnXHJcbiAgXHRcdH1cclxuICBcdH0sXHJcblxyXG4gIFx0YmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xyXG5cclxuICB9O1xyXG5cclxuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoSGlnaGNoYXJ0cy50aGVtZSk7XHJcblxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxyXG4gICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cclxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXHJcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxyXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgIG1haW46XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5
 nLWpvYnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcclxuICBcclxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxyXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgbWFpbjpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxyXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcclxuICAgIHVybDogXCJcIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIixcclxuICAgIHVybDogXCJc
 IlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0Lm92ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLFxyXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcclxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInIFxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXHJcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxyXG4gICAgdXJsOiBcIi97dmVydGV4SWR9XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICB2ZXJ0ZXg6XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4L
 mh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLFxyXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IuZXhjZXB0aW9uc1wiLFxyXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIixcclxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOl
 xyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiYWxsLW1hbmFnZXJcIixcclxuICAgIHVybDogXCIvdGFza21hbmFnZXJzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL2luZGV4Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL3Rhc2ttYW5hZ2VyL3t0YXNrbWFuYWdlcmlkfVwiXHJcbiAgICAgIHZpZXdzOlxyXG4gICAgICAgIG1haW46XHJcbiAgICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci

<TRUNCATED>

[05/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
new file mode 100644
index 0000000..660f8bc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
@@ -0,0 +1,379 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.PathChildrenCache;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.ConcurrentModificationException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * {@link SubmittedJobGraph} instances for JobManagers running in {@link RecoveryMode#ZOOKEEPER}.
+ *
+ * <p>Each job graph creates ZNode:
+ * <pre>
+ * +----O /flink/jobgraphs/&lt;job-id&gt; 1 [persistent]
+ * .
+ * .
+ * .
+ * +----O /flink/jobgraphs/&lt;job-id&gt; N [persistent]
+ * </pre>
+ *
+ * <p>The root path is watched to detect concurrent modifications in corner situations where
+ * multiple instances operate concurrently. The job manager acts as a {@link SubmittedJobGraphListener}
+ * to react to such situations.
+ */
+public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperSubmittedJobGraphStore.class);
+
+	/** Lock to synchronize with the {@link SubmittedJobGraphListener}. */
+	private final Object cacheLock = new Object();
+
+	/** Client (not a namespace facade) */
+	private final CuratorFramework client;
+
+	/** The set of IDs of all added job graphs. */
+	private final Set<JobID> addedJobGraphs = new HashSet<>();
+
+	/** Completed checkpoints in ZooKeeper */
+	private final ZooKeeperStateHandleStore<SubmittedJobGraph> jobGraphsInZooKeeper;
+
+	/**
+	 * Cache to monitor all children. This is used to detect races with other instances working
+	 * on the same state.
+	 */
+	private final PathChildrenCache pathCache;
+
+	/** The external listener to be notified on races. */
+	private SubmittedJobGraphListener jobGraphListener;
+
+	/** Flag indicating whether this instance is running. */
+	private boolean isRunning;
+
+	public ZooKeeperSubmittedJobGraphStore(
+			CuratorFramework client,
+			String currentJobsPath,
+			StateHandleProvider<SubmittedJobGraph> stateHandleProvider) throws Exception {
+
+		checkNotNull(currentJobsPath, "Current jobs path");
+		checkNotNull(stateHandleProvider, "State handle provider");
+
+		// Keep a reference to the original client and not the namespace facade. The namespace
+		// facade cannot be closed.
+		this.client = checkNotNull(client, "Curator client");
+
+		// Ensure that the job graphs path exists
+		client.newNamespaceAwareEnsurePath(currentJobsPath)
+				.ensure(client.getZookeeperClient());
+
+		// All operations will have the path as root
+		client = client.usingNamespace(client.getNamespace() + currentJobsPath);
+
+		this.jobGraphsInZooKeeper = new ZooKeeperStateHandleStore<>(client, stateHandleProvider);
+
+		this.pathCache = new PathChildrenCache(client, "/", false);
+		pathCache.getListenable().addListener(new SubmittedJobGraphsPathCacheListener());
+	}
+
+	@Override
+	public void start(SubmittedJobGraphListener jobGraphListener) throws Exception {
+		synchronized (cacheLock) {
+			if (!isRunning) {
+				this.jobGraphListener = jobGraphListener;
+
+				pathCache.start();
+
+				isRunning = true;
+			}
+		}
+	}
+
+	@Override
+	public void stop() throws Exception {
+		synchronized (cacheLock) {
+			if (isRunning) {
+				jobGraphListener = null;
+
+				pathCache.close();
+
+				client.close();
+
+				isRunning = false;
+			}
+		}
+	}
+
+	@Override
+	public List<SubmittedJobGraph> recoverJobGraphs() throws Exception {
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			List<Tuple2<StateHandle<SubmittedJobGraph>, String>> submitted;
+
+			while (true) {
+				try {
+					submitted = jobGraphsInZooKeeper.getAll();
+					break;
+				}
+				catch (ConcurrentModificationException e) {
+					LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying.");
+				}
+			}
+
+			if (submitted.size() != 0) {
+				List<SubmittedJobGraph> jobGraphs = new ArrayList<>(submitted.size());
+
+				for (Tuple2<StateHandle<SubmittedJobGraph>, String> jobStateHandle : submitted) {
+					SubmittedJobGraph jobGraph = jobStateHandle
+							.f0.getState(ClassLoader.getSystemClassLoader());
+
+					addedJobGraphs.add(jobGraph.getJobId());
+
+					jobGraphs.add(jobGraph);
+				}
+
+				LOG.info("Recovered {} job graphs: {}.", jobGraphs.size(), jobGraphs);
+				return jobGraphs;
+			}
+			else {
+				LOG.info("No job graph to recover.");
+				return Collections.emptyList();
+			}
+		}
+	}
+
+	@Override
+	public Option<SubmittedJobGraph> recoverJobGraph(JobID jobId) throws Exception {
+		checkNotNull(jobId, "Job ID");
+		String path = getPathForJob(jobId);
+
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			try {
+				StateHandle<SubmittedJobGraph> jobStateHandle = jobGraphsInZooKeeper.get(path);
+
+				SubmittedJobGraph jobGraph = jobStateHandle
+						.getState(ClassLoader.getSystemClassLoader());
+
+				addedJobGraphs.add(jobGraph.getJobId());
+
+				LOG.info("Recovered {}.", jobGraph);
+
+				return Option.apply(jobGraph);
+			}
+			catch (KeeperException.NoNodeException ignored) {
+				return Option.empty();
+			}
+		}
+	}
+
+	@Override
+	public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception {
+		checkNotNull(jobGraph, "Job graph");
+		String path = getPathForJob(jobGraph.getJobId());
+
+		boolean success = false;
+
+		while (!success) {
+			synchronized (cacheLock) {
+				verifyIsRunning();
+
+				int currentVersion = jobGraphsInZooKeeper.exists(path);
+
+				if (currentVersion == -1) {
+					try {
+						jobGraphsInZooKeeper.add(path, jobGraph);
+
+						addedJobGraphs.add(jobGraph.getJobId());
+
+						LOG.info("Added {} to ZooKeeper.", jobGraph);
+
+						success = true;
+					}
+					catch (KeeperException.NodeExistsException ignored) {
+					}
+				}
+				else if (addedJobGraphs.contains(jobGraph.getJobId())) {
+					try {
+						jobGraphsInZooKeeper.replace(path, currentVersion, jobGraph);
+						LOG.info("Updated {} in ZooKeeper.", jobGraph);
+
+						success = true;
+					}
+					catch (KeeperException.NoNodeException ignored) {
+					}
+				}
+				else {
+					throw new IllegalStateException("Oh, no. Trying to update a graph you didn't " +
+							"#getAllSubmittedJobGraphs() or #putJobGraph() yourself before.");
+				}
+			}
+		}
+	}
+
+	@Override
+	public void removeJobGraph(JobID jobId) throws Exception {
+		checkNotNull(jobId, "Job ID");
+		String path = getPathForJob(jobId);
+
+		synchronized (cacheLock) {
+			if (addedJobGraphs.contains(jobId)) {
+				jobGraphsInZooKeeper.removeAndDiscardState(path);
+
+				addedJobGraphs.remove(jobId);
+				LOG.info("Removed job graph {} from ZooKeeper.", jobId);
+			}
+		}
+	}
+
+	/**
+	 * Monitors ZooKeeper for changes.
+	 *
+	 * <p>Detects modifications from other job managers in corner situations. The event
+	 * notifications fire for changes from this job manager as well.
+	 */
+	private final class SubmittedJobGraphsPathCacheListener implements PathChildrenCacheListener {
+
+		@Override
+		public void childEvent(CuratorFramework client, PathChildrenCacheEvent event)
+				throws Exception {
+
+			if (LOG.isDebugEnabled()) {
+				if (event.getData() != null) {
+					LOG.debug("Received {} event (path: {})", event.getType(), event.getData().getPath());
+				}
+				else {
+					LOG.debug("Received {} event", event.getType());
+				}
+			}
+
+			switch (event.getType()) {
+				case CHILD_ADDED:
+					synchronized (cacheLock) {
+						try {
+							JobID jobId = fromEvent(event);
+							if (jobGraphListener != null && !addedJobGraphs.contains(jobId)) {
+								try {
+									// Whoa! This has been added by someone else. Or we were fast
+									// to remove it (false positive).
+									jobGraphListener.onAddedJobGraph(jobId);
+								}
+								catch (Throwable t) {
+									LOG.error("Error in callback", t);
+								}
+							}
+						}
+						catch (Exception e) {
+							LOG.error("Error in SubmittedJobGraphsPathCacheListener", e);
+						}
+					}
+
+					break;
+
+				case CHILD_UPDATED:
+					// Nothing to do
+					break;
+
+				case CHILD_REMOVED:
+					synchronized (cacheLock) {
+						try {
+							JobID jobId = fromEvent(event);
+							if (jobGraphListener != null && addedJobGraphs.contains(jobId)) {
+								try {
+									// Oh oh. Someone else removed one of our job graphs. Mean!
+									jobGraphListener.onRemovedJobGraph(jobId);
+								}
+								catch (Throwable t) {
+									LOG.error("Error in callback", t);
+								}
+							}
+
+							break;
+						}
+						catch (Exception e) {
+							LOG.error("Error in SubmittedJobGraphsPathCacheListener", e);
+						}
+					}
+					break;
+
+				case CONNECTION_SUSPENDED:
+					LOG.warn("ZooKeeper connection SUSPENDED. Changes to the submitted job " +
+							"graphs are not monitored (temporarily).");
+
+				case CONNECTION_LOST:
+					LOG.warn("ZooKeeper connection LOST. Changes to the submitted job " +
+							"graphs are not monitored (permanently).");
+					break;
+
+				case CONNECTION_RECONNECTED:
+					LOG.info("ZooKeeper connection RECONNECTED. Changes to the submitted job " +
+							"graphs are monitored again.");
+
+				case INITIALIZED:
+					LOG.info("SubmittedJobGraphsPathCacheListener initialized");
+					break;
+			}
+		}
+
+		/**
+		 * Returns a JobID for the event's path.
+		 */
+		private JobID fromEvent(PathChildrenCacheEvent event) {
+			return JobID.fromHexString(ZKPaths.getNodeFromPath(event.getData().getPath()));
+		}
+	}
+
+	/**
+	 * Verifies that the state is running.
+	 */
+	private void verifyIsRunning() {
+		checkState(isRunning, "Not running. Forgot to call start()?");
+	}
+
+	/**
+	 * Returns the JobID as a String (with leading slash).
+	 */
+	public static String getPathForJob(JobID jobId) {
+		checkNotNull(jobId, "Job ID");
+		return String.format("/%s", jobId);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
index b6223ee..6cba141 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
@@ -67,4 +67,5 @@ public interface LeaderElectionService {
 	 * @return true if the associated {@link LeaderContender} is the leader, otherwise false
 	 */
 	boolean hasLeadership();
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
index ae3f0e6..811037c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
@@ -43,6 +43,7 @@ import java.util.UUID;
  * ZooKeeper as well.
  */
 public class ZooKeeperLeaderElectionService implements LeaderElectionService, LeaderLatchListener, NodeCacheListener {
+
 	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionService.class);
 
 	/** Client to the ZooKeeper quorum */

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
new file mode 100644
index 0000000..7aa1ccf
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
@@ -0,0 +1,39 @@
+/*
+ * 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.state;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+
+public enum StateBackend {
+	JOBMANAGER, FILESYSTEM;
+
+	/**
+	 * Returns the configured {@link StateBackend}.
+	 *
+	 * @param config The config to parse
+	 * @return Configured state backend or {@link ConfigConstants#DEFAULT_RECOVERY_MODE} if not
+	 * configured.
+	 */
+	public static StateBackend fromConfig(Configuration config) {
+		return StateBackend.valueOf(config.getString(
+				ConfigConstants.STATE_BACKEND,
+				ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
new file mode 100644
index 0000000..0086ac6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.state;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+
+import java.io.Serializable;
+
+/**
+ * State handler provider factory.
+ *
+ * <p>This is going to be superseded soon.
+ */
+public class StateHandleProviderFactory {
+
+	/**
+	 * Creates a {@link org.apache.flink.runtime.state.FileStateHandle.FileStateHandleProvider} at
+	 * the configured recovery path.
+	 */
+	public static <T extends Serializable> StateHandleProvider<T> createRecoveryFileStateHandleProvider(
+			Configuration config) {
+
+		StateBackend stateBackend = StateBackend.fromConfig(config);
+
+		if (stateBackend == StateBackend.FILESYSTEM) {
+			String recoveryPath = config.getString(
+					ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
+
+			if (recoveryPath.equals("")) {
+				throw new IllegalConfigurationException("Missing recovery path. Specify via " +
+						"configuration key '" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "'.");
+			}
+			else {
+				return FileStateHandle.createProvider(recoveryPath);
+			}
+		}
+		else {
+			throw new IllegalConfigurationException("Unexpected state backend configuration " +
+					stateBackend);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
deleted file mode 100644
index 5f867a5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.util;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobmanager.RecoveryMode;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
-
-/**
- * Utility class to help working with {@link LeaderElectionService} class.
- */
-public final class LeaderElectionUtils {
-
-	/**
-	 * Creates a {@link LeaderElectionService} based on the provided {@link Configuration} object.
-	 *
-	 * @param configuration Configuration object
-	 * @return {@link LeaderElectionService} which was created based on the provided Configuration
-	 * @throws Exception
-	 */
-	public static LeaderElectionService createLeaderElectionService(Configuration configuration) throws Exception {
-		RecoveryMode recoveryMode = RecoveryMode.valueOf(configuration.getString(
-				ConfigConstants.RECOVERY_MODE,
-				ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase()
-		);
-
-		LeaderElectionService leaderElectionService;
-
-		switch(recoveryMode) {
-			case STANDALONE:
-				leaderElectionService = new StandaloneLeaderElectionService();
-				break;
-			case ZOOKEEPER:
-				leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
-				break;
-			default:
-				throw new Exception("Unknown RecoveryMode " + recoveryMode);
-		}
-
-		return leaderElectionService;
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private LeaderElectionUtils() {
-		throw new RuntimeException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
index d2d3945..79b9b7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
@@ -21,19 +21,27 @@ package org.apache.flink.runtime.util;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore;
 import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
+import org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.runtime.state.StateHandleProviderFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Utility class to work with Apache Zookeeper for Flink runtime.
- */
-public final class ZooKeeperUtils {
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class ZooKeeperUtils {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtils.class);
 
@@ -47,8 +55,10 @@ public final class ZooKeeperUtils {
 	public static CuratorFramework startCuratorFramework(Configuration configuration) {
 		String zkQuorum = configuration.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, "");
 
-		if(zkQuorum == null || zkQuorum.equals("")) {
-			throw new RuntimeException("No valid ZooKeeper quorum has been specified.");
+		if (zkQuorum == null || zkQuorum.equals("")) {
+			throw new RuntimeException("No valid ZooKeeper quorum has been specified. " +
+					"You can specify the quorum via the configuration key '" +
+					ConfigConstants.ZOOKEEPER_QUORUM_KEY + "'.");
 		}
 
 		int sessionTimeout = configuration.getInteger(
@@ -59,7 +69,7 @@ public final class ZooKeeperUtils {
 				ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT,
 				ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT);
 
-		int retryWait = configuration.getInteger (
+		int retryWait = configuration.getInteger(
 				ConfigConstants.ZOOKEEPER_RETRY_WAIT,
 				ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT);
 
@@ -88,14 +98,10 @@ public final class ZooKeeperUtils {
 	}
 
 	/**
-	 * Returns whether high availability is enabled (<=> ZooKeeper quorum configured).
+	 * Returns whether {@link RecoveryMode#ZOOKEEPER} is configured.
 	 */
-	public static boolean isZooKeeperHighAvailabilityEnabled(Configuration flinkConf) {
-		String recoveryMode = flinkConf.getString(
-				ConfigConstants.RECOVERY_MODE,
-				ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase();
-
-		return recoveryMode.equals(RecoveryMode.ZOOKEEPER.name());
+	public static boolean isZooKeeperRecoveryMode(Configuration flinkConf) {
+		return RecoveryMode.fromConfig(flinkConf).equals(RecoveryMode.ZOOKEEPER);
 	}
 
 	/**
@@ -125,7 +131,7 @@ public final class ZooKeeperUtils {
 	 * @throws Exception
 	 */
 	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration) throws Exception{
+			Configuration configuration) throws Exception {
 		CuratorFramework client = startCuratorFramework(configuration);
 		String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
 				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
@@ -134,7 +140,8 @@ public final class ZooKeeperUtils {
 	}
 
 	/**
-	 * Creates a {@link ZooKeeperLeaderElectionService} instance.
+	 * Creates a {@link ZooKeeperLeaderElectionService} instance and a new {@link
+	 * CuratorFramework} client.
 	 *
 	 * @param configuration {@link Configuration} object containing the configuration values
 	 * @return {@link ZooKeeperLeaderElectionService} instance.
@@ -142,8 +149,24 @@ public final class ZooKeeperUtils {
 	 */
 	public static ZooKeeperLeaderElectionService createLeaderElectionService(
 			Configuration configuration) throws Exception {
+
 		CuratorFramework client = startCuratorFramework(configuration);
 
+		return createLeaderElectionService(client, configuration);
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderElectionService} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link ZooKeeperLeaderElectionService} instance.
+	 * @throws Exception
+	 */
+	public static ZooKeeperLeaderElectionService createLeaderElectionService(
+			CuratorFramework client,
+			Configuration configuration) throws Exception {
+
 		String latchPath = configuration.getString(ConfigConstants.ZOOKEEPER_LATCH_PATH,
 				ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH);
 		String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
@@ -153,6 +176,89 @@ public final class ZooKeeperUtils {
 	}
 
 	/**
+	 * Creates a {@link ZooKeeperSubmittedJobGraphStore} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration {@link Configuration} object
+	 * @return {@link ZooKeeperSubmittedJobGraphStore} instance
+	 */
+	public static ZooKeeperSubmittedJobGraphStore createSubmittedJobGraphs(
+			CuratorFramework client,
+			Configuration configuration) throws Exception {
+
+		checkNotNull(configuration, "Configuration");
+
+		StateHandleProvider<SubmittedJobGraph> stateHandleProvider =
+				StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration);
+
+		// ZooKeeper submitted jobs root dir
+		String zooKeeperSubmittedJobsPath = configuration.getString(
+				ConfigConstants.ZOOKEEPER_JOBGRAPHS_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH);
+
+		return new ZooKeeperSubmittedJobGraphStore(
+				client, zooKeeperSubmittedJobsPath, stateHandleProvider);
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperCompletedCheckpointStore} instance.
+	 *
+	 * @param client                         The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration                  {@link Configuration} object
+	 * @param jobId                          ID of job to create the instance for
+	 * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain
+	 * @param userClassLoader                User code class loader
+	 * @return {@link ZooKeeperCompletedCheckpointStore} instance
+	 */
+	public static CompletedCheckpointStore createCompletedCheckpoints(
+			CuratorFramework client,
+			Configuration configuration,
+			JobID jobId,
+			int maxNumberOfCheckpointsToRetain,
+			ClassLoader userClassLoader) throws Exception {
+
+		checkNotNull(configuration, "Configuration");
+
+		StateHandleProvider<CompletedCheckpoint> stateHandleProvider =
+				StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration);
+
+		String completedCheckpointsPath = configuration.getString(
+				ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH);
+
+		completedCheckpointsPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId);
+
+		return new ZooKeeperCompletedCheckpointStore(
+				maxNumberOfCheckpointsToRetain,
+				userClassLoader,
+				client,
+				completedCheckpointsPath,
+				stateHandleProvider);
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperCheckpointIDCounter} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration {@link Configuration} object
+	 * @param jobId         ID of job to create the instance for
+	 * @return {@link ZooKeeperCheckpointIDCounter} instance
+	 */
+	public static ZooKeeperCheckpointIDCounter createCheckpointIDCounter(
+			CuratorFramework client,
+			Configuration configuration,
+			JobID jobId) throws Exception {
+
+		String checkpointIdCounterPath = configuration.getString(
+				ConfigConstants.ZOOKEEPER_CHECKPOINT_COUNTER_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH);
+
+		checkpointIdCounterPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId);
+
+		return new ZooKeeperCheckpointIDCounter(client, checkpointIdCounterPath);
+	}
+
+	/**
 	 * Private constructor to prevent instantiation.
 	 */
 	private ZooKeeperUtils() {

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
new file mode 100644
index 0000000..936fe1b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
@@ -0,0 +1,384 @@
+/*
+ * 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.zookeeper;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * State handles backed by ZooKeeper.
+ *
+ * <p>Added state is persisted via {@link StateHandle}s, which in turn are written to
+ * ZooKeeper. This level of indirection is necessary to keep the amount of data in ZooKeeper
+ * small. ZooKeeper is build for data in the KB range whereas state can grow to multiple MBs.
+ *
+ * <p>State modifications require some care, because it is possible that certain failures bring
+ * the state handle backend and ZooKeeper out of sync.
+ *
+ * <p>ZooKeeper holds the ground truth about state handles, i.e. the following holds:
+ *
+ * <pre>
+ * State handle in ZooKeeper => State handle exists
+ * </pre>
+ *
+ * But not:
+ *
+ * <pre>
+ * State handle exists => State handle in ZooKeeper
+ * </pre>
+ *
+ * There can be lingering state handles when failures happen during operation. They
+ * need to be cleaned up manually (see <a href="https://issues.apache.org/jira/browse/FLINK-2513">
+ * FLINK-2513</a> about a possible way to overcome this).
+ *
+ * @param <T> Type of state
+ */
+public class ZooKeeperStateHandleStore<T extends Serializable> {
+
+	/** Curator ZooKeeper client */
+	private final CuratorFramework client;
+
+	/** State handle provider */
+	private final StateHandleProvider<T> stateHandleProvider;
+
+	/**
+	 * Creates a {@link ZooKeeperStateHandleStore}.
+	 *
+	 * @param client              The Curator ZooKeeper client. <strong>Important:</strong> It is
+	 *                            expected that the client's namespace ensures that the root
+	 *                            path is exclusive for all state handles managed by this
+	 *                            instance, e.g. <code>client.usingNamespace("/stateHandles")</code>
+	 * @param stateHandleProvider The state handle provider for the state
+	 */
+	public ZooKeeperStateHandleStore(
+			CuratorFramework client,
+			StateHandleProvider<T> stateHandleProvider) {
+
+		this.client = checkNotNull(client, "Curator client");
+		this.stateHandleProvider = checkNotNull(stateHandleProvider, "State handle provider");
+	}
+
+	/**
+	 * Creates a state handle and stores it in ZooKeeper with create mode {@link
+	 * CreateMode#PERSISTENT}.
+	 *
+	 * @see #add(String, Serializable, CreateMode)
+	 */
+	public StateHandle<T> add(String pathInZooKeeper, T state) throws Exception {
+		return add(pathInZooKeeper, state, CreateMode.PERSISTENT);
+	}
+
+	/**
+	 * Creates a state handle and stores it in ZooKeeper.
+	 *
+	 * <p><strong>Important</strong>: This will <em>not</em> store the actual state in
+	 * ZooKeeper, but create a state handle and store it in ZooKeeper. This level of indirection
+	 * makes sure that data in ZooKeeper is small.
+	 *
+	 * @param pathInZooKeeper Destination path in ZooKeeper (expected to *not* exist yet and
+	 *                        start with a '/')
+	 * @param state           State to be added
+	 * @param createMode      The create mode for the new path in ZooKeeper
+	 * @return Created {@link StateHandle}
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	public StateHandle<T> add(String pathInZooKeeper, T state, CreateMode createMode) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+		checkNotNull(state, "State");
+
+		// Create the state handle. Nothing persisted yet.
+		StateHandle<T> stateHandle = stateHandleProvider.createStateHandle(state);
+
+		boolean success = false;
+
+		try {
+			// Serialize the state handle. This writes the state to the backend.
+			byte[] serializedStateHandle = InstantiationUtil.serializeObject(stateHandle);
+
+			// Write state handle (not the actual state) to ZooKeeper. This is expected to be
+			// smaller than the state itself. This level of indirection makes sure that data in
+			// ZooKeeper is small, because ZooKeeper is designed for data in the KB range, but
+			// the state can be larger.
+			client.create().withMode(createMode).forPath(pathInZooKeeper, serializedStateHandle);
+
+			success = true;
+
+			return stateHandle;
+		}
+		finally {
+			if (!success) {
+				// Cleanup the state handle if it was not written to ZooKeeper.
+				if (stateHandle != null) {
+					stateHandle.discardState();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Replaces a state handle in ZooKeeper and discards the old state handle.
+	 *
+	 * @param pathInZooKeeper Destination path in ZooKeeper (expected to exist and start with a '/')
+	 * @param expectedVersion Expected version of the node to replace
+	 * @param state           The new state to replace the old one
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	public void replace(String pathInZooKeeper, int expectedVersion, T state) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+		checkNotNull(state, "State");
+
+		StateHandle<T> oldStateHandle = get(pathInZooKeeper);
+
+		StateHandle<T> stateHandle = stateHandleProvider.createStateHandle(state);
+
+		boolean success = false;
+
+		try {
+			// Serialize the new state handle. This writes the state to the backend.
+			byte[] serializedStateHandle = InstantiationUtil.serializeObject(stateHandle);
+
+			// Replace state handle in ZooKeeper.
+			client.setData()
+					.withVersion(expectedVersion)
+					.forPath(pathInZooKeeper, serializedStateHandle);
+
+			success = true;
+		}
+		finally {
+			if (success) {
+				oldStateHandle.discardState();
+			}
+			else {
+				stateHandle.discardState();
+			}
+		}
+	}
+
+	/**
+	 * Returns the version of the node if it exists or <code>-1</code> if it doesn't.
+	 *
+	 * @param pathInZooKeeper Path in ZooKeeper to check
+	 * @return Version of the ZNode if the path exists, <code>-1</code> otherwise.
+	 * @throws Exception If the ZooKeeper operation fails
+	 */
+	public int exists(String pathInZooKeeper) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+
+		Stat stat = client.checkExists().forPath(pathInZooKeeper);
+
+		if (stat != null) {
+			return stat.getVersion();
+		}
+
+		return -1;
+	}
+
+	/**
+	 * Gets a state handle from ZooKeeper.
+	 *
+	 * @param pathInZooKeeper Path in ZooKeeper to get the state handle from (expected to
+	 *                        exist and start with a '/').
+	 * @return The state handle
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	@SuppressWarnings("unchecked")
+	public StateHandle<T> get(String pathInZooKeeper) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+
+		byte[] data = client.getData().forPath(pathInZooKeeper);
+
+		return (StateHandle<T>) InstantiationUtil
+				.deserializeObject(data, ClassLoader.getSystemClassLoader());
+	}
+
+	/**
+	 * Gets all available state handles from ZooKeeper.
+	 *
+	 * <p>If there is a concurrent modification, the operation is retried until it succeeds.
+	 *
+	 * @return All state handles from ZooKeeper.
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	@SuppressWarnings("unchecked")
+	public List<Tuple2<StateHandle<T>, String>> getAll() throws Exception {
+		final List<Tuple2<StateHandle<T>, String>> stateHandles = new ArrayList<>();
+
+		boolean success = false;
+
+		retry:
+		while (!success) {
+			// Initial cVersion (number of changes to the children of this node)
+			int initialCVersion = client.checkExists().forPath("/").getCversion();
+
+			List<String> children = client.getChildren().forPath("/");
+
+			for (String path : children) {
+				path = "/" + path;
+
+				try {
+					final StateHandle<T> stateHandle = get(path);
+					stateHandles.add(new Tuple2<>(stateHandle, path));
+				}
+				catch (KeeperException.NoNodeException ignored) {
+					// Concurrent deletion, retry
+					continue retry;
+				}
+			}
+
+			int finalCVersion = client.checkExists().forPath("/").getCversion();
+
+			// Check for concurrent modifications
+			success = initialCVersion == finalCVersion;
+		}
+
+		return stateHandles;
+	}
+
+	/**
+	 * Gets all available state handles from ZooKeeper sorted by name (ascending).
+	 *
+	 * <p>If there is a concurrent modification, the operation is retried until it succeeds.
+	 *
+	 * @return All state handles in ZooKeeper.
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	@SuppressWarnings("unchecked")
+	public List<Tuple2<StateHandle<T>, String>> getAllSortedByName() throws Exception {
+		final List<Tuple2<StateHandle<T>, String>> stateHandles = new ArrayList<>();
+
+		boolean success = false;
+
+		retry:
+		while (!success) {
+			// Initial cVersion (number of changes to the children of this node)
+			int initialCVersion = client.checkExists().forPath("/").getCversion();
+
+			List<String> children = ZKPaths.getSortedChildren(
+					client.getZookeeperClient().getZooKeeper(),
+					ZKPaths.fixForNamespace(client.getNamespace(), "/"));
+
+			for (String path : children) {
+				path = "/" + path;
+
+				try {
+					final StateHandle<T> stateHandle = get(path);
+					stateHandles.add(new Tuple2<>(stateHandle, path));
+				}
+				catch (KeeperException.NoNodeException ignored) {
+					// Concurrent deletion, retry
+					continue retry;
+				}
+			}
+
+			int finalCVersion = client.checkExists().forPath("/").getCversion();
+
+			// Check for concurrent modifications
+			success = initialCVersion == finalCVersion;
+		}
+
+		return stateHandles;
+	}
+
+	/**
+	 * Removes a state handle from ZooKeeper.
+	 *
+	 * <p><stong>Important</stong>: this does not discard the state handle. If you want to
+	 * discard the state handle call {@link #removeAndDiscardState(String)}.
+	 *
+	 * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/')
+	 * @throws Exception If the ZooKeeper operation fails
+	 */
+	public void remove(String pathInZooKeeper) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+
+		client.delete().deletingChildrenIfNeeded().forPath(pathInZooKeeper);
+	}
+
+	/**
+	 * Removes a state handle from ZooKeeper asynchronously.
+	 *
+	 * <p><stong>Important</stong>: this does not discard the state handle. If you want to
+	 * discard the state handle call {@link #removeAndDiscardState(String)}.
+	 *
+	 * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/')
+	 * @param callback        The callback after the operation finishes
+	 * @throws Exception If the ZooKeeper operation fails
+	 */
+	public void remove(String pathInZooKeeper, BackgroundCallback callback) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+		checkNotNull(callback, "Background callback");
+
+		client.delete().deletingChildrenIfNeeded().inBackground(callback).forPath(pathInZooKeeper);
+	}
+
+	/**
+	 * Discards a state handle and removes it from ZooKeeper.
+	 *
+	 * <p>If you only want to remove the state handle in ZooKeeper call {@link #remove(String)}.
+	 *
+	 * @param pathInZooKeeper Path of state handle to discard (expected to start with a '/')
+	 * @throws Exception If the ZooKeeper or state handle operation fails
+	 */
+	public void removeAndDiscardState(String pathInZooKeeper) throws Exception {
+		checkNotNull(pathInZooKeeper, "Path in ZooKeeper");
+
+		StateHandle<T> stateHandle = get(pathInZooKeeper);
+
+		// Delete the state handle from ZooKeeper first
+		client.delete().deletingChildrenIfNeeded().forPath(pathInZooKeeper);
+
+		// Discard the state handle only after it has been successfully deleted from ZooKeeper.
+		// Otherwise we might enter an illegal state after failures (with a state handle in
+		// ZooKeeper, which has already been discarded).
+		stateHandle.discardState();
+	}
+
+	/**
+	 * Discards all available state handles and removes them from ZooKeeper.
+	 *
+	 * @throws Exception If a ZooKeeper or state handle operation fails
+	 */
+	public void removeAndDiscardAllState() throws Exception {
+		final List<Tuple2<StateHandle<T>, String>> allStateHandles = getAll();
+
+		ZKPaths.deleteChildren(
+				client.getZookeeperClient().getZooKeeper(),
+				ZKPaths.fixForNamespace(client.getNamespace(), "/"),
+				false);
+
+		// Discard the state handles only after they have been successfully deleted from ZooKeeper.
+		for (Tuple2<StateHandle<T>, String> stateHandleAndPath : allStateHandles) {
+			stateHandleAndPath.f0.discardState();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala
index 75ad20f..67d7a06 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.jobmanager
 
 import akka.actor.ActorRef
-
+import org.apache.flink.runtime.akka.ListeningBehaviour
 
 /**
  * Utility class to store job information on the [[JobManager]]. The JobInfo stores which actor
@@ -27,11 +27,20 @@ import akka.actor.ActorRef
  * Additionally, it stores whether the job was started in the detached mode. Detached means that
  * the submitting actor does not wait for the job result once the job has terminated.
  *
+ * Important: This class is serializable, but needs to be deserialized in the context of an actor
+ * system in order to resolve the client [[ActorRef]]. It is possible to serialize the Akka URL
+ * manually, but it is cumbersome and complicates testing in certain scenarios, where you need to
+ * make sure to resolve the correct [[ActorRef]]s when submitting jobs (RepointableActorRef vs.
+ * RemoteActorRef).
+ *
  * @param client Actor which submitted the job
  * @param start Starting time
  */
-class JobInfo(val client: ActorRef, val start: Long,
-              val sessionTimeout: Long) {
+class JobInfo(
+  val client: ActorRef,
+  val listeningBehaviour: ListeningBehaviour,
+  val start: Long,
+  val sessionTimeout: Long) extends Serializable {
 
   var sessionAlive = sessionTimeout > 0
 
@@ -49,12 +58,16 @@ class JobInfo(val client: ActorRef, val start: Long,
     }
   }
 
+  override def toString = s"JobInfo(client: $client ($listeningBehaviour), start: $start)"
+
   def setLastActive() =
     lastActive = System.currentTimeMillis()
 }
 
 object JobInfo{
-  def apply(client: ActorRef, start: Long,
-            sessionTimeout: Long) =
-    new JobInfo(client, start, sessionTimeout)
+  def apply(
+    client: ActorRef,
+    listeningBehaviour: ListeningBehaviour,
+    start: Long,
+    sessionTimeout: Long) = new JobInfo(client, listeningBehaviour, start, sessionTimeout)
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/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 95637bb..f3e4054 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
@@ -19,31 +19,39 @@
 package org.apache.flink.runtime.jobmanager
 
 import java.io.{File, IOException}
-import java.lang.reflect.{InvocationTargetException, Constructor}
+import java.lang.reflect.{Constructor, InvocationTargetException}
 import java.net.InetSocketAddress
 import java.util.UUID
 
 import akka.actor.Status.Failure
-import akka.actor.{Props, Terminated, PoisonPill, ActorRef, ActorSystem}
+import akka.actor._
 import akka.pattern.ask
-
 import grizzled.slf4j.Logger
-
 import org.apache.flink.api.common.{ExecutionConfig, JobID}
 import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
 import org.apache.flink.core.io.InputSplitAssigner
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
+import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
 import org.apache.flink.runtime.blob.BlobServer
+import org.apache.flink.runtime.checkpoint.{CheckpointRecoveryFactory, StandaloneCheckpointRecoveryFactory, ZooKeeperCheckpointRecoveryFactory}
 import org.apache.flink.runtime.client._
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex}
+import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator
-import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener
+import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
+import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
+import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
+import org.apache.flink.runtime.messages.RegistrationMessages._
+import org.apache.flink.runtime.messages.TaskManagerMessages.{Heartbeat, SendStackTrace}
 import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState}
-import org.apache.flink.runtime.messages.accumulators._
+import org.apache.flink.runtime.messages.accumulators.{AccumulatorMessage, AccumulatorResultStringsFound, AccumulatorResultsErroneous, AccumulatorResultsFound, RequestAccumulatorResults, RequestAccumulatorResultsStringified}
 import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint}
 import org.apache.flink.runtime.messages.webmonitor._
 import org.apache.flink.runtime.process.ProcessReaper
@@ -51,25 +59,16 @@ import org.apache.flink.runtime.security.SecurityUtils
 import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.util._
-import org.apache.flink.runtime.webmonitor.{WebMonitorUtils, WebMonitor}
-import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessageFilter}
-import org.apache.flink.runtime.LogMessages
-import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils}
-import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
-import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
-import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus}
-import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
-import org.apache.flink.runtime.messages.JobManagerMessages._
-import org.apache.flink.runtime.messages.RegistrationMessages._
-import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat}
-import org.apache.flink.util.{NetUtils, SerializedValue, ExceptionUtils, InstantiationUtil}
+import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils}
+import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages, StreamingMode}
+import org.apache.flink.util.{ExceptionUtils, InstantiationUtil, NetUtils}
 
+import scala.collection.JavaConverters._
+import scala.concurrent.ExecutionContext.Implicits.global
 import scala.concurrent._
 import scala.concurrent.duration._
 import scala.concurrent.forkjoin.ForkJoinPool
 import scala.language.postfixOps
-import scala.collection.JavaConverters._
-import scala.concurrent.ExecutionContext.Implicits.global
 
 
 /**
@@ -110,17 +109,22 @@ class JobManager(
     protected val delayBetweenRetries: Long,
     protected val timeout: FiniteDuration,
     protected val mode: StreamingMode,
-    protected val leaderElectionService: LeaderElectionService)
+    protected val leaderElectionService: LeaderElectionService,
+    protected val submittedJobGraphs : SubmittedJobGraphStore,
+    protected val checkpointRecoveryFactory : CheckpointRecoveryFactory)
   extends FlinkActor 
   with LeaderSessionMessageFilter // mixin oder is important, we want filtering after logging
   with LogMessages // mixin order is important, we want first logging
-  with LeaderContender {
+  with LeaderContender
+  with SubmittedJobGraphListener {
 
   override val log = Logger(getClass)
 
   /** Either running or not yet archived jobs (session hasn't been ended). */
   protected val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]()
 
+  protected val recoveryMode = RecoveryMode.fromConfig(flinkConfiguration)
+
   var leaderSessionID: Option[UUID] = None
 
   /**
@@ -138,6 +142,22 @@ class JobManager(
           "start.", e)
         throw new RuntimeException("Could not start the leader election service.", e)
     }
+
+    try {
+      submittedJobGraphs.start(this)
+    } catch {
+      case e: Exception =>
+        log.error("Could not start the submitted job graphs service.", e)
+        throw new RuntimeException("Could not start the submitted job graphs service.", e)
+    }
+
+    try {
+      checkpointRecoveryFactory.start()
+    } catch {
+      case e: Exception =>
+        log.error("Could not start the checkpoint recovery service.", e)
+        throw new RuntimeException("Could not start the checkpoint recovery service.", e)
+    }
   }
 
   override def postStop(): Unit = {
@@ -159,6 +179,18 @@ class JobManager(
       case e: Exception => log.error("Could not properly shutdown the leader election service.")
     }
 
+    try {
+      submittedJobGraphs.stop()
+    } catch {
+      case e: Exception => log.error("Could not properly stop the submitted job graphs service.")
+    }
+
+    try {
+      checkpointRecoveryFactory.stop()
+    } catch {
+      case e: Exception => log.error("Could not properly stop the checkpoint recovery service.")
+    }
+
     if (archive != ActorRef.noSender) {
       archive ! decorateMessage(PoisonPill)
     }
@@ -191,12 +223,21 @@ class JobManager(
       // confirming the leader session ID might be blocking, thus do it in a future
       future{
         leaderElectionService.confirmLeaderSessionID(newLeaderSessionID.orNull)
+
+        // TODO (critical next step) This needs to be more flexible and robust (e.g. wait for task
+        // managers etc.)
+        if (recoveryMode != RecoveryMode.STANDALONE) {
+          context.system.scheduler.scheduleOnce(new FiniteDuration(delayBetweenRetries,
+            MILLISECONDS), self, decorateMessage(RecoverAllJobs))(context.dispatcher)
+        }
       }(context.dispatcher)
 
     case RevokeLeadership =>
       log.info(s"JobManager ${self.path.toSerializationFormat} was revoked leadership.")
 
-      cancelAndClearEverything(new Exception("JobManager is no longer the leader."))
+      future {
+        cancelAndClearEverything(new Exception("JobManager is no longer the leader."))
+      }(context.dispatcher)
 
       // disconnect the registered task managers
       instanceManager.getAllRegisteredInstances.asScala.foreach {
@@ -269,7 +310,62 @@ class JobManager(
       sender ! decorateMessage(instanceManager.getTotalNumberOfSlots)
 
     case SubmitJob(jobGraph, listeningBehaviour) =>
-      submitJob(jobGraph, listeningBehaviour)
+      val client = sender()
+
+      val jobInfo = new JobInfo(client, listeningBehaviour, System.currentTimeMillis(),
+        jobGraph.getSessionTimeout)
+
+      future {
+        submitJob(jobGraph, jobInfo)
+      }(context.dispatcher)
+
+    case RecoverJob(jobId) =>
+      future {
+        // The ActorRef, which is part of the submitted job graph can only be deserialized in the
+        // scope of an actor system.
+        akka.serialization.JavaSerializer.currentSystem.withValue(
+          context.system.asInstanceOf[ExtendedActorSystem]) {
+
+          log.info(s"Attempting to recover job $jobId.")
+
+          val jobGraph = submittedJobGraphs.recoverJobGraph(jobId)
+
+          if (jobGraph.isDefined) {
+            if (!leaderElectionService.hasLeadership()) {
+              // we've lost leadership. mission: abort.
+              log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.")
+            }
+            else {
+              recoverJobGraph(jobGraph.get)
+            }
+          }
+          else {
+            log.warn(s"Failed to recover job graph ${jobId}.")
+          }
+        }
+      }(context.dispatcher)
+
+    case RecoverAllJobs =>
+      future {
+        // The ActorRef, which is part of the submitted job graph can only be deserialized in the
+        // scope of an actor system.
+        akka.serialization.JavaSerializer.currentSystem.withValue(
+          context.system.asInstanceOf[ExtendedActorSystem]) {
+
+          val jobGraphs = submittedJobGraphs.recoverJobGraphs().asScala
+
+          if (!leaderElectionService.hasLeadership()) {
+            // we've lost leadership. mission: abort.
+            log.warn(s"Lost leadership during recovery. Aborting recovery of ${jobGraphs.size} " +
+              s"jobs.")
+          }
+          else {
+            log.debug(s"Attempting to recover ${jobGraphs.size} job graphs.")
+
+            jobGraphs.foreach(recoverJobGraph(_))
+          }
+        }
+      }(context.dispatcher)
 
     case CancelJob(jobID) =>
       log.info(s"Trying to cancel job with ID $jobID.")
@@ -377,10 +473,27 @@ class JobManager(
           if (newJobStatus.isTerminalState()) {
             jobInfo.end = timeStamp
 
-            // is the client waiting for the job result?
-            if (jobInfo.client != ActorRef.noSender) {
-              newJobStatus match {
-                case JobStatus.FINISHED =>
+            future {
+              // TODO If removing the JobGraph from the SubmittedJobGraphsStore fails, the job will
+              // linger around and potentially be recovered at a later time. There is nothing we
+              // can do about that, but it should be communicated with the Client.
+              if (jobInfo.sessionAlive) {
+                jobInfo.setLastActive()
+                val lastActivity = jobInfo.lastActive
+                context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) {
+                  // remove only if no activity occurred in the meantime
+                  if (lastActivity == jobInfo.lastActive) {
+                    removeJob(jobID)
+                  }
+                }
+              } else {
+                removeJob(jobID)
+              }
+
+              // is the client waiting for the job result?
+              if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) {
+                newJobStatus match {
+                  case JobStatus.FINISHED =>
                   try {
                     val accumulatorResults = executionGraph.getAccumulatorsSerialized()
                     val result = new SerializedJobExecutionResult(
@@ -398,47 +511,37 @@ class JobManager(
                       jobInfo.client ! decorateMessage(JobResultFailure(
                         new SerializedThrowable(exception)))
                   }
-                case JobStatus.CANCELED =>
-                  // the error may be packed as a serialized throwable
-                  val unpackedError = SerializedThrowable.get(
-                    error, executionGraph.getUserClassLoader())
-
-                  jobInfo.client ! decorateMessage(JobResultFailure(
-                    new SerializedThrowable(
-                      new JobCancellationException(jobID, "Job was cancelled.", unpackedError))))
-
-                case JobStatus.FAILED =>
-                  val unpackedError = SerializedThrowable.get(
-                    error, executionGraph.getUserClassLoader())
-
-                  jobInfo.client ! decorateMessage(JobResultFailure(
-                    new SerializedThrowable(
-                      new JobExecutionException(jobID, "Job execution failed.", unpackedError))))
-
-                case x =>
-                  val exception = new JobExecutionException(jobID, s"$x is not a terminal state.")
-                  jobInfo.client ! decorateMessage(JobResultFailure(
-                    new SerializedThrowable(exception)))
-                  throw exception
-              }
-            }
 
-            if (jobInfo.sessionAlive) {
-              jobInfo.setLastActive()
-              val lastActivity = jobInfo.lastActive
-              context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) {
-                // remove only if no activity occurred in the meantime
-                if (lastActivity == jobInfo.lastActive) {
-                  removeJob(jobID)
+                  case JobStatus.CANCELED =>
+                    // the error may be packed as a serialized throwable
+                    val unpackedError = SerializedThrowable.get(
+                      error, executionGraph.getUserClassLoader())
+
+                    jobInfo.client ! decorateMessage(JobResultFailure(
+                      new SerializedThrowable(
+                        new JobCancellationException(jobID, "Job was cancelled.", unpackedError))))
+
+                  case JobStatus.FAILED =>
+                    val unpackedError = SerializedThrowable.get(
+                      error, executionGraph.getUserClassLoader())
+
+                    jobInfo.client ! decorateMessage(JobResultFailure(
+                      new SerializedThrowable(
+                        new JobExecutionException(jobID, "Job execution failed.", unpackedError))))
+
+                  case x =>
+                    val exception = new JobExecutionException(jobID, s"$x is not a terminal state.")
+                    jobInfo.client ! decorateMessage(JobResultFailure(
+                      new SerializedThrowable(exception)))
+                    throw exception
                 }
               }
-            } else {
-              removeJob(jobID)
-            }
-
+            }(context.dispatcher)
           }
         case None =>
-          removeJob(jobID)
+          future {
+            removeJob(jobID)
+          }(context.dispatcher)
       }
 
     case ScheduleOrUpdateConsumers(jobId, partitionId) =>
@@ -600,11 +703,12 @@ class JobManager(
    * graph and the execution vertices are queued for scheduling.
    *
    * @param jobGraph representing the Flink job
-   * @param listeningBehaviour specifies the listening behaviour of the sender.
+   * @param jobInfo the job info
+   * @param isRecovery Flag indicating whether this is a recovery or initial submission
    */
-  private def submitJob(jobGraph: JobGraph, listeningBehaviour: ListeningBehaviour): Unit = {
+  private def submitJob(jobGraph: JobGraph, jobInfo: JobInfo, isRecovery: Boolean = false): Unit = {
     if (jobGraph == null) {
-      sender() ! decorateMessage(JobResultFailure(
+      jobInfo.client ! decorateMessage(JobResultFailure(
         new SerializedThrowable(
           new JobSubmissionException(null, "JobGraph must not be null.")
         )
@@ -615,7 +719,7 @@ class JobManager(
       val jobName = jobGraph.getName
       var executionGraph: ExecutionGraph = null
 
-      log.info(s"Received job ${jobId} (${jobName}).")
+      log.info(s"Submitting job $jobId ($jobName)" + (if (isRecovery) " (Recovery)" else "") + ".")
 
       try {
         // Important: We need to make sure that the library registration is the first action,
@@ -628,7 +732,7 @@ class JobManager(
         catch {
           case t: Throwable =>
             throw new JobSubmissionException(jobId,
-            "Cannot set up the user code libraries: " + t.getMessage, t)
+              "Cannot set up the user code libraries: " + t.getMessage, t)
         }
 
         val userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID)
@@ -641,18 +745,10 @@ class JobManager(
           throw new JobSubmissionException(jobId, "The given job is empty")
         }
 
-        val client = if(listeningBehaviour == ListeningBehaviour.DETACHED) {
-          // The client does not want to receive the SerializedJobExecutionResult
-          ActorRef.noSender
-        } else {
-          // Send the job execution result back to the sender
-          sender
-        }
-
         // see if there already exists an ExecutionGraph for the corresponding job ID
         executionGraph = currentJobs.get(jobGraph.getJobID) match {
-          case Some((graph, jobInfo)) =>
-            jobInfo.setLastActive()
+          case Some((graph, currentJobInfo)) =>
+            currentJobInfo.setLastActive()
             graph
           case None =>
             val graph = new ExecutionGraph(
@@ -664,11 +760,7 @@ class JobManager(
               jobGraph.getUserJarBlobKeys,
               jobGraph.getClasspaths,
               userCodeLoader)
-            val jobInfo = JobInfo(
-              client,
-              System.currentTimeMillis(),
-              jobGraph.getSessionTimeout)
-            currentJobs.put(jobGraph.getJobID, (graph, jobInfo))
+
             graph
         }
 
@@ -682,7 +774,7 @@ class JobManager(
         executionGraph.setDelayBeforeRetrying(delayBetweenRetries)
         executionGraph.setScheduleMode(jobGraph.getScheduleMode())
         executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling())
-        
+
         try {
           executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph))
         }
@@ -691,7 +783,7 @@ class JobManager(
             log.warn("Cannot create JSON plan for job", t)
             executionGraph.setJsonPlan("{}")
         }
-        
+
         // initialize the vertices that have a master initialization hook
         // file output formats create directories here, input formats create splits
         if (log.isDebugEnabled) {
@@ -701,62 +793,67 @@ class JobManager(
         val numSlots = scheduler.getTotalNumberOfSlots()
 
         for (vertex <- jobGraph.getVertices.asScala) {
-
           val executableClass = vertex.getInvokableClassName
           if (executableClass == null || executableClass.length == 0) {
             throw new JobSubmissionException(jobId,
               s"The vertex ${vertex.getID} (${vertex.getName}) has no invokable class.")
           }
 
-              if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-                vertex.setParallelism(numSlots)
-              }
+          if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
+            vertex.setParallelism(numSlots)
+          }
 
-              try {
-                vertex.initializeOnMaster(userCodeLoader)
-              }
-              catch {
+          try {
+            vertex.initializeOnMaster(userCodeLoader)
+          }
+          catch {
             case t: Throwable =>
               throw new JobExecutionException(jobId,
                 "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage, t)
-              }
-            }
+          }
+        }
 
-            // topologically sort the job vertices and attach the graph to the existing one
-            val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources()
-            if (log.isDebugEnabled) {
-              log.debug(s"Adding ${sortedTopology.size()} vertices from " +
-                s"job graph ${jobId} (${jobName}).")
-            }
-            executionGraph.attachJobGraph(sortedTopology)
+        // topologically sort the job vertices and attach the graph to the existing one
+        val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources()
+        if (log.isDebugEnabled) {
+          log.debug(s"Adding ${sortedTopology.size()} vertices from " +
+            s"job graph ${jobId} (${jobName}).")
+        }
+        executionGraph.attachJobGraph(sortedTopology)
 
-            if (log.isDebugEnabled) {
-              log.debug("Successfully created execution graph from job " +
-                s"graph ${jobId} (${jobName}).")
-            }
+        if (log.isDebugEnabled) {
+          log.debug("Successfully created execution graph from job " +
+            s"graph ${jobId} (${jobName}).")
+        }
 
-            // configure the state checkpointing
-            val snapshotSettings = jobGraph.getSnapshotSettings
-            if (snapshotSettings != null) {
+        // configure the state checkpointing
+        val snapshotSettings = jobGraph.getSnapshotSettings
+        if (snapshotSettings != null) {
+          val jobId = jobGraph.getJobID()
 
-              val idToVertex: JobVertexID => ExecutionJobVertex = id => {
-                val vertex = executionGraph.getJobVertex(id)
-                if (vertex == null) {
-                  throw new JobSubmissionException(jobId,
-                    "The snapshot checkpointing settings refer to non-existent vertex " + id)
-                }
-                vertex
-              }
+          val idToVertex: JobVertexID => ExecutionJobVertex = id => {
+            val vertex = executionGraph.getJobVertex(id)
+            if (vertex == null) {
+              throw new JobSubmissionException(jobId,
+                "The snapshot checkpointing settings refer to non-existent vertex " + id)
+            }
+            vertex
+          }
 
-              val triggerVertices: java.util.List[ExecutionJobVertex] =
+          val triggerVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToTrigger().asScala.map(idToVertex).asJava
 
-              val ackVertices: java.util.List[ExecutionJobVertex] =
+          val ackVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToAcknowledge().asScala.map(idToVertex).asJava
 
-              val confirmVertices: java.util.List[ExecutionJobVertex] =
+          val confirmVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToConfirm().asScala.map(idToVertex).asJava
 
+          val completedCheckpoints = checkpointRecoveryFactory
+            .createCompletedCheckpoints(jobId, userCodeLoader)
+
+          val checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(jobId)
+
           executionGraph.enableSnapshotCheckpointing(
             snapshotSettings.getCheckpointInterval,
             snapshotSettings.getCheckpointTimeout,
@@ -764,23 +861,39 @@ class JobManager(
             ackVertices,
             confirmVertices,
             context.system,
-            leaderSessionID.orNull)
+            leaderSessionID.orNull,
+            checkpointIdCounter,
+            completedCheckpoints,
+            recoveryMode)
         }
 
         // get notified about job status changes
         executionGraph.registerJobStatusListener(
           new AkkaActorGateway(self, leaderSessionID.orNull))
 
-        if (listeningBehaviour == ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES) {
+        if (jobInfo.listeningBehaviour == ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES) {
           // the sender wants to be notified about state changes
-          val gateway = new AkkaActorGateway(sender(), leaderSessionID.orNull)
+          val gateway = new AkkaActorGateway(jobInfo.client, leaderSessionID.orNull)
 
           executionGraph.registerExecutionListener(gateway)
           executionGraph.registerJobStatusListener(gateway)
         }
 
+        if (isRecovery) {
+          executionGraph.restoreLatestCheckpointedState()
+        }
+        else {
+          submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo))
+        }
+
+        // Add the job graph only after everything is finished. Otherwise there can be races in
+        // tests, which check the currentJobs (for example before killing a JM).
+        if (!currentJobs.contains(jobId)) {
+          currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo))
+        }
+
         // done with submitting the job
-        sender() ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID))
+        jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID))
       }
       catch {
         case t: Throwable =>
@@ -799,33 +912,61 @@ class JobManager(
             new JobExecutionException(jobId, s"Failed to submit job ${jobId} (${jobName})", t)
           }
 
-          sender() ! decorateMessage(JobResultFailure(new SerializedThrowable(rt)))
+          jobInfo.client ! decorateMessage(JobResultFailure(new SerializedThrowable(rt)))
           return
       }
 
-      // NOTE: Scheduling the job for execution is a separate action from the job submission.
-      // The success of submitting the job must be independent from the success of scheduling
-      // the job.
-      try {
-        log.info(s"Scheduling job ${executionGraph.getJobName}.")
-        executionGraph.scheduleForExecution(scheduler)
-      }
-      catch {
-        case t: Throwable => try {
-          executionGraph.fail(t)
+      if (leaderElectionService.hasLeadership) {
+        // There is a small chance that multiple job managers schedule the same job after if they
+        // try to recover at the same time. This will eventually be noticed, but can not be ruled
+        // out from the beginning.
+
+        // NOTE: Scheduling the job for execution is a separate action from the job submission.
+        // The success of submitting the job must be independent from the success of scheduling
+        // the job.
+        try {
+          log.info(s"Scheduling job $jobId ($jobName).")
+
+          executionGraph.scheduleForExecution(scheduler)
         }
         catch {
-          case tt: Throwable => {
-            log.error("Error while marking ExecutionGraph as failed.", tt)
+          case t: Throwable => try {
+            executionGraph.fail(t)
+          }
+          catch {
+            case tt: Throwable => {
+              log.error("Error while marking ExecutionGraph as failed.", tt)
+            }
           }
         }
       }
+      else {
+        // Remove the job graph. Otherwise it will be lingering around and possibly removed from
+        // ZooKeeper by this JM.
+        currentJobs.remove(jobId)
+
+        log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " +
+          "this. I am not scheduling the job for execution.")
+      }
+    }
+  }
+
+  /**
+   * Submits the job if it is not already one of our current jobs.
+   *
+   * @param jobGraph Job to recover
+   */
+  private def recoverJobGraph(jobGraph: SubmittedJobGraph): Unit = {
+    if (!currentJobs.contains(jobGraph.getJobId)) {
+      future {
+        submitJob(jobGraph.getJobGraph(), jobGraph.getJobInfo(), isRecovery = true)
+      }(context.dispatcher)
     }
   }
 
   /**
    * Dedicated handler for checkpoint messages.
-   * 
+   *
    * @param actorMessage The checkpoint actor message.
    */
   private def handleCheckpointMessage(actorMessage: AbstractCheckpointMessage): Unit = {
@@ -836,13 +977,15 @@ class JobManager(
           case Some((graph, _)) =>
             val coordinator = graph.getCheckpointCoordinator()
             if (coordinator != null) {
-              try {
-                coordinator.receiveAcknowledgeMessage(ackMessage)
-              }
-              catch {
-                case t: Throwable =>
-                  log.error(s"Error in CheckpointCoordinator while processing $ackMessage", t)
-              }
+              future {
+                try {
+                  coordinator.receiveAcknowledgeMessage(ackMessage)
+                }
+                catch {
+                  case t: Throwable =>
+                    log.error(s"Error in CheckpointCoordinator while processing $ackMessage", t)
+                }
+              }(context.dispatcher)
             }
             else {
               log.error(
@@ -1020,30 +1163,46 @@ class JobManager(
   }
 
   /**
-   * Removes the job and sends it to the MemoryArchivist
+   * Removes the job and sends it to the MemoryArchivist.
+   *
+   * This should be called asynchronously. Removing the job from the [[SubmittedJobGraphStore]]
+   * might block. Therefore be careful not to block the actor thread.
+   *
    * @param jobID ID of the job to remove and archive
    */
   private def removeJob(jobID: JobID): Unit = {
     currentJobs.synchronized {
-      currentJobs.remove(jobID) match {
+      // Don't remove the job yet...
+      currentJobs.get(jobID) match {
         case Some((eg, _)) =>
           try {
+            // ...otherwise, we can have lingering resources when there is a  concurrent shutdown
+            // and the ZooKeeper client is closed. Not removing the job immediately allow the
+            // shutdown to release all resources.
+            submittedJobGraphs.removeJobGraph(jobID)
+          } catch {
+            case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t)
+          }
+
+          try {
             eg.prepareForArchiving()
+
             archive ! decorateMessage(ArchiveExecutionGraph(jobID, eg))
           } catch {
             case t: Throwable => log.error(s"Could not prepare the execution graph $eg for " +
               "archiving.", t)
           }
 
+          currentJobs.remove(jobID)
         case None =>
       }
+    }
 
-      try {
-        libraryCacheManager.unregisterJob(jobID)
-      } catch {
-        case t: Throwable =>
-          log.error(s"Could not properly unregister job $jobID form the library cache.", t)
-      }
+    try {
+      libraryCacheManager.unregisterJob(jobID)
+    } catch {
+      case t: Throwable =>
+        log.error(s"Could not properly unregister job $jobID form the library cache.", t)
     }
   }
 
@@ -1053,17 +1212,21 @@ class JobManager(
     * @param cause Cause for the cancelling.
     */
   private def cancelAndClearEverything(cause: Throwable) {
-    for((jobID, (eg, jobInfo)) <- currentJobs) {
+    for ((jobID, (eg, jobInfo)) <- currentJobs) {
+      try {
+        submittedJobGraphs.removeJobGraph(jobID)
+      }
+      catch {
+        case t: Throwable => {
+          log.error("Error during submitted job graph clean up.", t)
+        }
+      }
+
       eg.fail(cause)
 
-      if(jobInfo.client != ActorRef.noSender) {
+      if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) {
         jobInfo.client ! decorateMessage(
-          Failure(
-            new JobExecutionException(
-              jobID,
-              "All jobs are cancelled and cleared.",
-              cause)
-          ))
+          Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause)))
       }
     }
 
@@ -1079,6 +1242,25 @@ class JobManager(
     self ! decorateMessage(RevokeLeadership)
   }
 
+  override def onAddedJobGraph(jobId: JobID): Unit = {
+    if (leaderSessionID.isDefined && !currentJobs.contains(jobId)) {
+      self ! decorateMessage(RecoverJob(jobId))
+    }
+  }
+
+  override def onRemovedJobGraph(jobId: JobID): Unit = {
+    if (leaderSessionID.isDefined) {
+      currentJobs.get(jobId).foreach(
+        job =>
+          future {
+            // Fail the execution graph
+            job._1.fail(new IllegalStateException("Another JobManager removed the job from " +
+              "ZooKeeper."))
+          }(context.dispatcher)
+      )
+    }
+  }
+
   override def getAddress: String = {
     AkkaUtils.getAkkaURL(context.system, self)
   }
@@ -1166,7 +1348,7 @@ object JobManager {
       System.exit(STARTUP_FAILURE_RETURN_CODE)
     }
 
-    if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) {
+    if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) {
       // address and will not be reachable from anyone remote
       if (listeningPort != 0) {
         val message = "Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY +
@@ -1227,7 +1409,7 @@ object JobManager {
    *
    * @param configuration The configuration object for the JobManager.
    * @param executionMode The execution mode in which to run. Execution mode LOCAL will spawn an
-   *                      additional TaskManager in the same process.
+   *                      an additional TaskManager in the same process.
    * @param streamingMode The streaming mode to run the system in (streaming vs. batch-only)
    * @param listeningAddress The hostname where the JobManager should listen for messages.
    * @param listeningPort The port where the JobManager should listen for messages.
@@ -1480,7 +1662,7 @@ object JobManager {
 
     // high availability mode
     val port: Int =
-      if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) {
+      if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) {
         LOG.info("Starting JobManager in High-Availability Mode")
 
         configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
@@ -1524,7 +1706,9 @@ object JobManager {
     Long, // delay between retries
     FiniteDuration, // timeout
     Int, // number of archived jobs
-    LeaderElectionService) = {
+    LeaderElectionService,
+    SubmittedJobGraphStore,
+    CheckpointRecoveryFactory) = {
 
     val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration)
 
@@ -1588,10 +1772,31 @@ object JobManager {
       }
     }
 
-    val leaderElectionService = leaderElectionServiceOption match {
-      case Some(les) => les
-      case None => LeaderElectionUtils.createLeaderElectionService(configuration)
-    }
+    // Create recovery related components
+    val (leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory) =
+      RecoveryMode.fromConfig(configuration) match {
+        case RecoveryMode.STANDALONE =>
+          val leaderElectionService = leaderElectionServiceOption match {
+            case Some(les) => les
+            case None => new StandaloneLeaderElectionService()
+          }
+
+          (leaderElectionService,
+            new StandaloneSubmittedJobGraphStore(),
+            new StandaloneCheckpointRecoveryFactory())
+
+        case RecoveryMode.ZOOKEEPER =>
+          val client = ZooKeeperUtils.startCuratorFramework(configuration)
+
+          val leaderElectionService = leaderElectionServiceOption match {
+            case Some(les) => les
+            case None => ZooKeeperUtils.createLeaderElectionService(client, configuration)
+          }
+
+          (leaderElectionService,
+            ZooKeeperUtils.createSubmittedJobGraphs(client, configuration),
+            new ZooKeeperCheckpointRecoveryFactory(client, configuration))
+      }
 
     (executionContext,
       instanceManager,
@@ -1599,9 +1804,11 @@ object JobManager {
       libraryCacheManager,
       executionRetries,
       delayBetweenRetries,
-      timeout, 
-      archiveCount, 
-      leaderElectionService)
+      timeout,
+      archiveCount,
+      leaderElectionService,
+      submittedJobGraphs,
+      checkpointRecoveryFactory)
   }
 
   /**
@@ -1633,6 +1840,7 @@ object JobManager {
       jobManagerClass,
       archiveClass)
   }
+
   /**
    * Starts the JobManager and job archiver based on the given configuration, in the
    * given actor system.
@@ -1646,28 +1854,30 @@ object JobManager {
    * @param streamingMode The mode to run the system in (streaming vs. batch-only)
    * @param jobManagerClass The class of the JobManager to be started
    * @param archiveClass The class of the MemoryArchivist to be started
-   * 
+   *
    * @return A tuple of references (JobManager Ref, Archiver Ref)
    */
   def startJobManagerActors(
-      configuration: Configuration,
-      actorSystem: ActorSystem,
-      jobMangerActorName: Option[String],
-      archiveActorName: Option[String],
-      streamingMode: StreamingMode,
-      jobManagerClass: Class[_ <: JobManager],
-      archiveClass: Class[_ <: MemoryArchivist])
-    : (ActorRef, ActorRef) = {
+                             configuration: Configuration,
+                             actorSystem: ActorSystem,
+                             jobMangerActorName: Option[String],
+                             archiveActorName: Option[String],
+                             streamingMode: StreamingMode,
+                             jobManagerClass: Class[_ <: JobManager],
+                             archiveClass: Class[_ <: MemoryArchivist])
+  : (ActorRef, ActorRef) = {
 
     val (executionContext,
-      instanceManager,
-      scheduler,
-      libraryCacheManager,
-      executionRetries,
-      delayBetweenRetries,
-      timeout,
-      archiveCount,
-      leaderElectionService) = createJobManagerComponents(
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    executionRetries,
+    delayBetweenRetries,
+    timeout,
+    archiveCount,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory) = createJobManagerComponents(
       configuration,
       None)
 
@@ -1691,7 +1901,9 @@ object JobManager {
       delayBetweenRetries,
       timeout,
       streamingMode,
-      leaderElectionService)
+      leaderElectionService,
+      submittedJobGraphs,
+      checkpointRecoveryFactory)
 
     val jobManager: ActorRef = jobMangerActorName match {
       case Some(actorName) => actorSystem.actorOf(jobManagerProps, actorName)

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index c29df88..d776622 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -66,6 +66,18 @@ object JobManagerMessages {
     extends RequiresLeaderSessionID
 
   /**
+   * Triggers the recovery of the job with the given ID.
+   *
+   * @param jobId ID of the job to recover
+   */
+  case class RecoverJob(jobId: JobID) extends RequiresLeaderSessionID
+
+  /**
+   * Triggers recovery of all available jobs.
+   */
+  case class RecoverAllJobs() extends RequiresLeaderSessionID
+
+  /**
    * Cancels a job with the given [[jobID]] at the JobManager. The result of the cancellation is
    * sent back to the sender as a [[CancellationResponse]] message.
    *
@@ -354,6 +366,10 @@ object JobManagerMessages {
   // --------------------------------------------------------------------------
   // Utility methods to allow simpler case object access from Java
   // --------------------------------------------------------------------------
+
+  def getRequestJobStatus(jobId : JobID) : AnyRef = {
+    RequestJobStatus(jobId)
+  }
   
   def getRequestNumberRegisteredTaskManager : AnyRef = {
     RequestNumberRegisteredTaskManager

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index 29add0e..2df3437 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -94,9 +94,7 @@ abstract class FlinkMiniCluster(
 
   implicit val timeout = AkkaUtils.getTimeout(userConfiguration)
 
-  val recoveryMode = RecoveryMode.valueOf(configuration.getString(
-    ConfigConstants.RECOVERY_MODE,
-    ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase)
+  val recoveryMode = RecoveryMode.fromConfig(configuration)
 
   val numJobManagers = getNumberOfJobManagers
 


[16/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/css/vendor.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/css/vendor.css b/flink-runtime-web/web-dashboard/web/css/vendor.css
deleted file mode 100644
index 672e07f..0000000
--- a/flink-runtime-web/web-dashboard/web/css/vendor.css
+++ /dev/null
@@ -1,9183 +0,0 @@
-/*!
- *  Font Awesome 4.3.0 by @davegandy - http://fontawesome.io - @fontawesome
- *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
- */
-/* FONT PATH
- * -------------------------- */
-@font-face {
-  font-family: 'FontAwesome';
-  src: url('../fonts/fontawesome-webfont.eot?v=4.3.0');
-  src: url('../fonts/fontawesome-webfont.eot?#iefix&v=4.3.0') format('embedded-opentype'), url('../fonts/fontawesome-webfont.woff2?v=4.3.0') format('woff2'), url('../fonts/fontawesome-webfont.woff?v=4.3.0') format('woff'), url('../fonts/fontawesome-webfont.ttf?v=4.3.0') format('truetype'), url('../fonts/fontawesome-webfont.svg?v=4.3.0#fontawesomeregular') format('svg');
-  font-weight: normal;
-  font-style: normal;
-}
-.fa {
-  display: inline-block;
-  font: normal normal normal 14px/1 FontAwesome;
-  font-size: inherit;
-  text-rendering: auto;
-  -webkit-font-smoothing: antialiased;
-  -moz-osx-font-smoothing: grayscale;
-  transform: translate(0, 0);
-}
-/* makes the font 33% larger relative to the icon container */
-.fa-lg {
-  font-size: 1.33333333em;
-  line-height: 0.75em;
-  vertical-align: -15%;
-}
-.fa-2x {
-  font-size: 2em;
-}
-.fa-3x {
-  font-size: 3em;
-}
-.fa-4x {
-  font-size: 4em;
-}
-.fa-5x {
-  font-size: 5em;
-}
-.fa-fw {
-  width: 1.28571429em;
-  text-align: center;
-}
-.fa-ul {
-  padding-left: 0;
-  margin-left: 2.14285714em;
-  list-style-type: none;
-}
-.fa-ul > li {
-  position: relative;
-}
-.fa-li {
-  position: absolute;
-  left: -2.14285714em;
-  width: 2.14285714em;
-  top: 0.14285714em;
-  text-align: center;
-}
-.fa-li.fa-lg {
-  left: -1.85714286em;
-}
-.fa-border {
-  padding: .2em .25em .15em;
-  border: solid 0.08em #eeeeee;
-  border-radius: .1em;
-}
-.pull-right {
-  float: right;
-}
-.pull-left {
-  float: left;
-}
-.fa.pull-left {
-  margin-right: .3em;
-}
-.fa.pull-right {
-  margin-left: .3em;
-}
-.fa-spin {
-  -webkit-animation: fa-spin 2s infinite linear;
-  animation: fa-spin 2s infinite linear;
-}
-.fa-pulse {
-  -webkit-animation: fa-spin 1s infinite steps(8);
-  animation: fa-spin 1s infinite steps(8);
-}
-@-webkit-keyframes fa-spin {
-  0% {
-    -webkit-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-  100% {
-    -webkit-transform: rotate(359deg);
-    transform: rotate(359deg);
-  }
-}
-@keyframes fa-spin {
-  0% {
-    -webkit-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-  100% {
-    -webkit-transform: rotate(359deg);
-    transform: rotate(359deg);
-  }
-}
-.fa-rotate-90 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=1);
-  -webkit-transform: rotate(90deg);
-  -ms-transform: rotate(90deg);
-  transform: rotate(90deg);
-}
-.fa-rotate-180 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2);
-  -webkit-transform: rotate(180deg);
-  -ms-transform: rotate(180deg);
-  transform: rotate(180deg);
-}
-.fa-rotate-270 {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=3);
-  -webkit-transform: rotate(270deg);
-  -ms-transform: rotate(270deg);
-  transform: rotate(270deg);
-}
-.fa-flip-horizontal {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);
-  -webkit-transform: scale(-1, 1);
-  -ms-transform: scale(-1, 1);
-  transform: scale(-1, 1);
-}
-.fa-flip-vertical {
-  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);
-  -webkit-transform: scale(1, -1);
-  -ms-transform: scale(1, -1);
-  transform: scale(1, -1);
-}
-:root .fa-rotate-90,
-:root .fa-rotate-180,
-:root .fa-rotate-270,
-:root .fa-flip-horizontal,
-:root .fa-flip-vertical {
-  filter: none;
-}
-.fa-stack {
-  position: relative;
-  display: inline-block;
-  width: 2em;
-  height: 2em;
-  line-height: 2em;
-  vertical-align: middle;
-}
-.fa-stack-1x,
-.fa-stack-2x {
-  position: absolute;
-  left: 0;
-  width: 100%;
-  text-align: center;
-}
-.fa-stack-1x {
-  line-height: inherit;
-}
-.fa-stack-2x {
-  font-size: 2em;
-}
-.fa-inverse {
-  color: #ffffff;
-}
-/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen
-   readers do not read off random characters that represent icons */
-.fa-glass:before {
-  content: "\f000";
-}
-.fa-music:before {
-  content: "\f001";
-}
-.fa-search:before {
-  content: "\f002";
-}
-.fa-envelope-o:before {
-  content: "\f003";
-}
-.fa-heart:before {
-  content: "\f004";
-}
-.fa-star:before {
-  content: "\f005";
-}
-.fa-star-o:before {
-  content: "\f006";
-}
-.fa-user:before {
-  content: "\f007";
-}
-.fa-film:before {
-  content: "\f008";
-}
-.fa-th-large:before {
-  content: "\f009";
-}
-.fa-th:before {
-  content: "\f00a";
-}
-.fa-th-list:before {
-  content: "\f00b";
-}
-.fa-check:before {
-  content: "\f00c";
-}
-.fa-remove:before,
-.fa-close:before,
-.fa-times:before {
-  content: "\f00d";
-}
-.fa-search-plus:before {
-  content: "\f00e";
-}
-.fa-search-minus:before {
-  content: "\f010";
-}
-.fa-power-off:before {
-  content: "\f011";
-}
-.fa-signal:before {
-  content: "\f012";
-}
-.fa-gear:before,
-.fa-cog:before {
-  content: "\f013";
-}
-.fa-trash-o:before {
-  content: "\f014";
-}
-.fa-home:before {
-  content: "\f015";
-}
-.fa-file-o:before {
-  content: "\f016";
-}
-.fa-clock-o:before {
-  content: "\f017";
-}
-.fa-road:before {
-  content: "\f018";
-}
-.fa-download:before {
-  content: "\f019";
-}
-.fa-arrow-circle-o-down:before {
-  content: "\f01a";
-}
-.fa-arrow-circle-o-up:before {
-  content: "\f01b";
-}
-.fa-inbox:before {
-  content: "\f01c";
-}
-.fa-play-circle-o:before {
-  content: "\f01d";
-}
-.fa-rotate-right:before,
-.fa-repeat:before {
-  content: "\f01e";
-}
-.fa-refresh:before {
-  content: "\f021";
-}
-.fa-list-alt:before {
-  content: "\f022";
-}
-.fa-lock:before {
-  content: "\f023";
-}
-.fa-flag:before {
-  content: "\f024";
-}
-.fa-headphones:before {
-  content: "\f025";
-}
-.fa-volume-off:before {
-  content: "\f026";
-}
-.fa-volume-down:before {
-  content: "\f027";
-}
-.fa-volume-up:before {
-  content: "\f028";
-}
-.fa-qrcode:before {
-  content: "\f029";
-}
-.fa-barcode:before {
-  content: "\f02a";
-}
-.fa-tag:before {
-  content: "\f02b";
-}
-.fa-tags:before {
-  content: "\f02c";
-}
-.fa-book:before {
-  content: "\f02d";
-}
-.fa-bookmark:before {
-  content: "\f02e";
-}
-.fa-print:before {
-  content: "\f02f";
-}
-.fa-camera:before {
-  content: "\f030";
-}
-.fa-font:before {
-  content: "\f031";
-}
-.fa-bold:before {
-  content: "\f032";
-}
-.fa-italic:before {
-  content: "\f033";
-}
-.fa-text-height:before {
-  content: "\f034";
-}
-.fa-text-width:before {
-  content: "\f035";
-}
-.fa-align-left:before {
-  content: "\f036";
-}
-.fa-align-center:before {
-  content: "\f037";
-}
-.fa-align-right:before {
-  content: "\f038";
-}
-.fa-align-justify:before {
-  content: "\f039";
-}
-.fa-list:before {
-  content: "\f03a";
-}
-.fa-dedent:before,
-.fa-outdent:before {
-  content: "\f03b";
-}
-.fa-indent:before {
-  content: "\f03c";
-}
-.fa-video-camera:before {
-  content: "\f03d";
-}
-.fa-photo:before,
-.fa-image:before,
-.fa-picture-o:before {
-  content: "\f03e";
-}
-.fa-pencil:before {
-  content: "\f040";
-}
-.fa-map-marker:before {
-  content: "\f041";
-}
-.fa-adjust:before {
-  content: "\f042";
-}
-.fa-tint:before {
-  content: "\f043";
-}
-.fa-edit:before,
-.fa-pencil-square-o:before {
-  content: "\f044";
-}
-.fa-share-square-o:before {
-  content: "\f045";
-}
-.fa-check-square-o:before {
-  content: "\f046";
-}
-.fa-arrows:before {
-  content: "\f047";
-}
-.fa-step-backward:before {
-  content: "\f048";
-}
-.fa-fast-backward:before {
-  content: "\f049";
-}
-.fa-backward:before {
-  content: "\f04a";
-}
-.fa-play:before {
-  content: "\f04b";
-}
-.fa-pause:before {
-  content: "\f04c";
-}
-.fa-stop:before {
-  content: "\f04d";
-}
-.fa-forward:before {
-  content: "\f04e";
-}
-.fa-fast-forward:before {
-  content: "\f050";
-}
-.fa-step-forward:before {
-  content: "\f051";
-}
-.fa-eject:before {
-  content: "\f052";
-}
-.fa-chevron-left:before {
-  content: "\f053";
-}
-.fa-chevron-right:before {
-  content: "\f054";
-}
-.fa-plus-circle:before {
-  content: "\f055";
-}
-.fa-minus-circle:before {
-  content: "\f056";
-}
-.fa-times-circle:before {
-  content: "\f057";
-}
-.fa-check-circle:before {
-  content: "\f058";
-}
-.fa-question-circle:before {
-  content: "\f059";
-}
-.fa-info-circle:before {
-  content: "\f05a";
-}
-.fa-crosshairs:before {
-  content: "\f05b";
-}
-.fa-times-circle-o:before {
-  content: "\f05c";
-}
-.fa-check-circle-o:before {
-  content: "\f05d";
-}
-.fa-ban:before {
-  content: "\f05e";
-}
-.fa-arrow-left:before {
-  content: "\f060";
-}
-.fa-arrow-right:before {
-  content: "\f061";
-}
-.fa-arrow-up:before {
-  content: "\f062";
-}
-.fa-arrow-down:before {
-  content: "\f063";
-}
-.fa-mail-forward:before,
-.fa-share:before {
-  content: "\f064";
-}
-.fa-expand:before {
-  content: "\f065";
-}
-.fa-compress:before {
-  content: "\f066";
-}
-.fa-plus:before {
-  content: "\f067";
-}
-.fa-minus:before {
-  content: "\f068";
-}
-.fa-asterisk:before {
-  content: "\f069";
-}
-.fa-exclamation-circle:before {
-  content: "\f06a";
-}
-.fa-gift:before {
-  content: "\f06b";
-}
-.fa-leaf:before {
-  content: "\f06c";
-}
-.fa-fire:before {
-  content: "\f06d";
-}
-.fa-eye:before {
-  content: "\f06e";
-}
-.fa-eye-slash:before {
-  content: "\f070";
-}
-.fa-warning:before,
-.fa-exclamation-triangle:before {
-  content: "\f071";
-}
-.fa-plane:before {
-  content: "\f072";
-}
-.fa-calendar:before {
-  content: "\f073";
-}
-.fa-random:before {
-  content: "\f074";
-}
-.fa-comment:before {
-  content: "\f075";
-}
-.fa-magnet:before {
-  content: "\f076";
-}
-.fa-chevron-up:before {
-  content: "\f077";
-}
-.fa-chevron-down:before {
-  content: "\f078";
-}
-.fa-retweet:before {
-  content: "\f079";
-}
-.fa-shopping-cart:before {
-  content: "\f07a";
-}
-.fa-folder:before {
-  content: "\f07b";
-}
-.fa-folder-open:before {
-  content: "\f07c";
-}
-.fa-arrows-v:before {
-  content: "\f07d";
-}
-.fa-arrows-h:before {
-  content: "\f07e";
-}
-.fa-bar-chart-o:before,
-.fa-bar-chart:before {
-  content: "\f080";
-}
-.fa-twitter-square:before {
-  content: "\f081";
-}
-.fa-facebook-square:before {
-  content: "\f082";
-}
-.fa-camera-retro:before {
-  content: "\f083";
-}
-.fa-key:before {
-  content: "\f084";
-}
-.fa-gears:before,
-.fa-cogs:before {
-  content: "\f085";
-}
-.fa-comments:before {
-  content: "\f086";
-}
-.fa-thumbs-o-up:before {
-  content: "\f087";
-}
-.fa-thumbs-o-down:before {
-  content: "\f088";
-}
-.fa-star-half:before {
-  content: "\f089";
-}
-.fa-heart-o:before {
-  content: "\f08a";
-}
-.fa-sign-out:before {
-  content: "\f08b";
-}
-.fa-linkedin-square:before {
-  content: "\f08c";
-}
-.fa-thumb-tack:before {
-  content: "\f08d";
-}
-.fa-external-link:before {
-  content: "\f08e";
-}
-.fa-sign-in:before {
-  content: "\f090";
-}
-.fa-trophy:before {
-  content: "\f091";
-}
-.fa-github-square:before {
-  content: "\f092";
-}
-.fa-upload:before {
-  content: "\f093";
-}
-.fa-lemon-o:before {
-  content: "\f094";
-}
-.fa-phone:before {
-  content: "\f095";
-}
-.fa-square-o:before {
-  content: "\f096";
-}
-.fa-bookmark-o:before {
-  content: "\f097";
-}
-.fa-phone-square:before {
-  content: "\f098";
-}
-.fa-twitter:before {
-  content: "\f099";
-}
-.fa-facebook-f:before,
-.fa-facebook:before {
-  content: "\f09a";
-}
-.fa-github:before {
-  content: "\f09b";
-}
-.fa-unlock:before {
-  content: "\f09c";
-}
-.fa-credit-card:before {
-  content: "\f09d";
-}
-.fa-rss:before {
-  content: "\f09e";
-}
-.fa-hdd-o:before {
-  content: "\f0a0";
-}
-.fa-bullhorn:before {
-  content: "\f0a1";
-}
-.fa-bell:before {
-  content: "\f0f3";
-}
-.fa-certificate:before {
-  content: "\f0a3";
-}
-.fa-hand-o-right:before {
-  content: "\f0a4";
-}
-.fa-hand-o-left:before {
-  content: "\f0a5";
-}
-.fa-hand-o-up:before {
-  content: "\f0a6";
-}
-.fa-hand-o-down:before {
-  content: "\f0a7";
-}
-.fa-arrow-circle-left:before {
-  content: "\f0a8";
-}
-.fa-arrow-circle-right:before {
-  content: "\f0a9";
-}
-.fa-arrow-circle-up:before {
-  content: "\f0aa";
-}
-.fa-arrow-circle-down:before {
-  content: "\f0ab";
-}
-.fa-globe:before {
-  content: "\f0ac";
-}
-.fa-wrench:before {
-  content: "\f0ad";
-}
-.fa-tasks:before {
-  content: "\f0ae";
-}
-.fa-filter:before {
-  content: "\f0b0";
-}
-.fa-briefcase:before {
-  content: "\f0b1";
-}
-.fa-arrows-alt:before {
-  content: "\f0b2";
-}
-.fa-group:before,
-.fa-users:before {
-  content: "\f0c0";
-}
-.fa-chain:before,
-.fa-link:before {
-  content: "\f0c1";
-}
-.fa-cloud:before {
-  content: "\f0c2";
-}
-.fa-flask:before {
-  content: "\f0c3";
-}
-.fa-cut:before,
-.fa-scissors:before {
-  content: "\f0c4";
-}
-.fa-copy:before,
-.fa-files-o:before {
-  content: "\f0c5";
-}
-.fa-paperclip:before {
-  content: "\f0c6";
-}
-.fa-save:before,
-.fa-floppy-o:before {
-  content: "\f0c7";
-}
-.fa-square:before {
-  content: "\f0c8";
-}
-.fa-navicon:before,
-.fa-reorder:before,
-.fa-bars:before {
-  content: "\f0c9";
-}
-.fa-list-ul:before {
-  content: "\f0ca";
-}
-.fa-list-ol:before {
-  content: "\f0cb";
-}
-.fa-strikethrough:before {
-  content: "\f0cc";
-}
-.fa-underline:before {
-  content: "\f0cd";
-}
-.fa-table:before {
-  content: "\f0ce";
-}
-.fa-magic:before {
-  content: "\f0d0";
-}
-.fa-truck:before {
-  content: "\f0d1";
-}
-.fa-pinterest:before {
-  content: "\f0d2";
-}
-.fa-pinterest-square:before {
-  content: "\f0d3";
-}
-.fa-google-plus-square:before {
-  content: "\f0d4";
-}
-.fa-google-plus:before {
-  content: "\f0d5";
-}
-.fa-money:before {
-  content: "\f0d6";
-}
-.fa-caret-down:before {
-  content: "\f0d7";
-}
-.fa-caret-up:before {
-  content: "\f0d8";
-}
-.fa-caret-left:before {
-  content: "\f0d9";
-}
-.fa-caret-right:before {
-  content: "\f0da";
-}
-.fa-columns:before {
-  content: "\f0db";
-}
-.fa-unsorted:before,
-.fa-sort:before {
-  content: "\f0dc";
-}
-.fa-sort-down:before,
-.fa-sort-desc:before {
-  content: "\f0dd";
-}
-.fa-sort-up:before,
-.fa-sort-asc:before {
-  content: "\f0de";
-}
-.fa-envelope:before {
-  content: "\f0e0";
-}
-.fa-linkedin:before {
-  content: "\f0e1";
-}
-.fa-rotate-left:before,
-.fa-undo:before {
-  content: "\f0e2";
-}
-.fa-legal:before,
-.fa-gavel:before {
-  content: "\f0e3";
-}
-.fa-dashboard:before,
-.fa-tachometer:before {
-  content: "\f0e4";
-}
-.fa-comment-o:before {
-  content: "\f0e5";
-}
-.fa-comments-o:before {
-  content: "\f0e6";
-}
-.fa-flash:before,
-.fa-bolt:before {
-  content: "\f0e7";
-}
-.fa-sitemap:before {
-  content: "\f0e8";
-}
-.fa-umbrella:before {
-  content: "\f0e9";
-}
-.fa-paste:before,
-.fa-clipboard:before {
-  content: "\f0ea";
-}
-.fa-lightbulb-o:before {
-  content: "\f0eb";
-}
-.fa-exchange:before {
-  content: "\f0ec";
-}
-.fa-cloud-download:before {
-  content: "\f0ed";
-}
-.fa-cloud-upload:before {
-  content: "\f0ee";
-}
-.fa-user-md:before {
-  content: "\f0f0";
-}
-.fa-stethoscope:before {
-  content: "\f0f1";
-}
-.fa-suitcase:before {
-  content: "\f0f2";
-}
-.fa-bell-o:before {
-  content: "\f0a2";
-}
-.fa-coffee:before {
-  content: "\f0f4";
-}
-.fa-cutlery:before {
-  content: "\f0f5";
-}
-.fa-file-text-o:before {
-  content: "\f0f6";
-}
-.fa-building-o:before {
-  content: "\f0f7";
-}
-.fa-hospital-o:before {
-  content: "\f0f8";
-}
-.fa-ambulance:before {
-  content: "\f0f9";
-}
-.fa-medkit:before {
-  content: "\f0fa";
-}
-.fa-fighter-jet:before {
-  content: "\f0fb";
-}
-.fa-beer:before {
-  content: "\f0fc";
-}
-.fa-h-square:before {
-  content: "\f0fd";
-}
-.fa-plus-square:before {
-  content: "\f0fe";
-}
-.fa-angle-double-left:before {
-  content: "\f100";
-}
-.fa-angle-double-right:before {
-  content: "\f101";
-}
-.fa-angle-double-up:before {
-  content: "\f102";
-}
-.fa-angle-double-down:before {
-  content: "\f103";
-}
-.fa-angle-left:before {
-  content: "\f104";
-}
-.fa-angle-right:before {
-  content: "\f105";
-}
-.fa-angle-up:before {
-  content: "\f106";
-}
-.fa-angle-down:before {
-  content: "\f107";
-}
-.fa-desktop:before {
-  content: "\f108";
-}
-.fa-laptop:before {
-  content: "\f109";
-}
-.fa-tablet:before {
-  content: "\f10a";
-}
-.fa-mobile-phone:before,
-.fa-mobile:before {
-  content: "\f10b";
-}
-.fa-circle-o:before {
-  content: "\f10c";
-}
-.fa-quote-left:before {
-  content: "\f10d";
-}
-.fa-quote-right:before {
-  content: "\f10e";
-}
-.fa-spinner:before {
-  content: "\f110";
-}
-.fa-circle:before {
-  content: "\f111";
-}
-.fa-mail-reply:before,
-.fa-reply:before {
-  content: "\f112";
-}
-.fa-github-alt:before {
-  content: "\f113";
-}
-.fa-folder-o:before {
-  content: "\f114";
-}
-.fa-folder-open-o:before {
-  content: "\f115";
-}
-.fa-smile-o:before {
-  content: "\f118";
-}
-.fa-frown-o:before {
-  content: "\f119";
-}
-.fa-meh-o:before {
-  content: "\f11a";
-}
-.fa-gamepad:before {
-  content: "\f11b";
-}
-.fa-keyboard-o:before {
-  content: "\f11c";
-}
-.fa-flag-o:before {
-  content: "\f11d";
-}
-.fa-flag-checkered:before {
-  content: "\f11e";
-}
-.fa-terminal:before {
-  content: "\f120";
-}
-.fa-code:before {
-  content: "\f121";
-}
-.fa-mail-reply-all:before,
-.fa-reply-all:before {
-  content: "\f122";
-}
-.fa-star-half-empty:before,
-.fa-star-half-full:before,
-.fa-star-half-o:before {
-  content: "\f123";
-}
-.fa-location-arrow:before {
-  content: "\f124";
-}
-.fa-crop:before {
-  content: "\f125";
-}
-.fa-code-fork:before {
-  content: "\f126";
-}
-.fa-unlink:before,
-.fa-chain-broken:before {
-  content: "\f127";
-}
-.fa-question:before {
-  content: "\f128";
-}
-.fa-info:before {
-  content: "\f129";
-}
-.fa-exclamation:before {
-  content: "\f12a";
-}
-.fa-superscript:before {
-  content: "\f12b";
-}
-.fa-subscript:before {
-  content: "\f12c";
-}
-.fa-eraser:before {
-  content: "\f12d";
-}
-.fa-puzzle-piece:before {
-  content: "\f12e";
-}
-.fa-microphone:before {
-  content: "\f130";
-}
-.fa-microphone-slash:before {
-  content: "\f131";
-}
-.fa-shield:before {
-  content: "\f132";
-}
-.fa-calendar-o:before {
-  content: "\f133";
-}
-.fa-fire-extinguisher:before {
-  content: "\f134";
-}
-.fa-rocket:before {
-  content: "\f135";
-}
-.fa-maxcdn:before {
-  content: "\f136";
-}
-.fa-chevron-circle-left:before {
-  content: "\f137";
-}
-.fa-chevron-circle-right:before {
-  content: "\f138";
-}
-.fa-chevron-circle-up:before {
-  content: "\f139";
-}
-.fa-chevron-circle-down:before {
-  content: "\f13a";
-}
-.fa-html5:before {
-  content: "\f13b";
-}
-.fa-css3:before {
-  content: "\f13c";
-}
-.fa-anchor:before {
-  content: "\f13d";
-}
-.fa-unlock-alt:before {
-  content: "\f13e";
-}
-.fa-bullseye:before {
-  content: "\f140";
-}
-.fa-ellipsis-h:before {
-  content: "\f141";
-}
-.fa-ellipsis-v:before {
-  content: "\f142";
-}
-.fa-rss-square:before {
-  content: "\f143";
-}
-.fa-play-circle:before {
-  content: "\f144";
-}
-.fa-ticket:before {
-  content: "\f145";
-}
-.fa-minus-square:before {
-  content: "\f146";
-}
-.fa-minus-square-o:before {
-  content: "\f147";
-}
-.fa-level-up:before {
-  content: "\f148";
-}
-.fa-level-down:before {
-  content: "\f149";
-}
-.fa-check-square:before {
-  content: "\f14a";
-}
-.fa-pencil-square:before {
-  content: "\f14b";
-}
-.fa-external-link-square:before {
-  content: "\f14c";
-}
-.fa-share-square:before {
-  content: "\f14d";
-}
-.fa-compass:before {
-  content: "\f14e";
-}
-.fa-toggle-down:before,
-.fa-caret-square-o-down:before {
-  content: "\f150";
-}
-.fa-toggle-up:before,
-.fa-caret-square-o-up:before {
-  content: "\f151";
-}
-.fa-toggle-right:before,
-.fa-caret-square-o-right:before {
-  content: "\f152";
-}
-.fa-euro:before,
-.fa-eur:before {
-  content: "\f153";
-}
-.fa-gbp:before {
-  content: "\f154";
-}
-.fa-dollar:before,
-.fa-usd:before {
-  content: "\f155";
-}
-.fa-rupee:before,
-.fa-inr:before {
-  content: "\f156";
-}
-.fa-cny:before,
-.fa-rmb:before,
-.fa-yen:before,
-.fa-jpy:before {
-  content: "\f157";
-}
-.fa-ruble:before,
-.fa-rouble:before,
-.fa-rub:before {
-  content: "\f158";
-}
-.fa-won:before,
-.fa-krw:before {
-  content: "\f159";
-}
-.fa-bitcoin:before,
-.fa-btc:before {
-  content: "\f15a";
-}
-.fa-file:before {
-  content: "\f15b";
-}
-.fa-file-text:before {
-  content: "\f15c";
-}
-.fa-sort-alpha-asc:before {
-  content: "\f15d";
-}
-.fa-sort-alpha-desc:before {
-  content: "\f15e";
-}
-.fa-sort-amount-asc:before {
-  content: "\f160";
-}
-.fa-sort-amount-desc:before {
-  content: "\f161";
-}
-.fa-sort-numeric-asc:before {
-  content: "\f162";
-}
-.fa-sort-numeric-desc:before {
-  content: "\f163";
-}
-.fa-thumbs-up:before {
-  content: "\f164";
-}
-.fa-thumbs-down:before {
-  content: "\f165";
-}
-.fa-youtube-square:before {
-  content: "\f166";
-}
-.fa-youtube:before {
-  content: "\f167";
-}
-.fa-xing:before {
-  content: "\f168";
-}
-.fa-xing-square:before {
-  content: "\f169";
-}
-.fa-youtube-play:before {
-  content: "\f16a";
-}
-.fa-dropbox:before {
-  content: "\f16b";
-}
-.fa-stack-overflow:before {
-  content: "\f16c";
-}
-.fa-instagram:before {
-  content: "\f16d";
-}
-.fa-flickr:before {
-  content: "\f16e";
-}
-.fa-adn:before {
-  content: "\f170";
-}
-.fa-bitbucket:before {
-  content: "\f171";
-}
-.fa-bitbucket-square:before {
-  content: "\f172";
-}
-.fa-tumblr:before {
-  content: "\f173";
-}
-.fa-tumblr-square:before {
-  content: "\f174";
-}
-.fa-long-arrow-down:before {
-  content: "\f175";
-}
-.fa-long-arrow-up:before {
-  content: "\f176";
-}
-.fa-long-arrow-left:before {
-  content: "\f177";
-}
-.fa-long-arrow-right:before {
-  content: "\f178";
-}
-.fa-apple:before {
-  content: "\f179";
-}
-.fa-windows:before {
-  content: "\f17a";
-}
-.fa-android:before {
-  content: "\f17b";
-}
-.fa-linux:before {
-  content: "\f17c";
-}
-.fa-dribbble:before {
-  content: "\f17d";
-}
-.fa-skype:before {
-  content: "\f17e";
-}
-.fa-foursquare:before {
-  content: "\f180";
-}
-.fa-trello:before {
-  content: "\f181";
-}
-.fa-female:before {
-  content: "\f182";
-}
-.fa-male:before {
-  content: "\f183";
-}
-.fa-gittip:before,
-.fa-gratipay:before {
-  content: "\f184";
-}
-.fa-sun-o:before {
-  content: "\f185";
-}
-.fa-moon-o:before {
-  content: "\f186";
-}
-.fa-archive:before {
-  content: "\f187";
-}
-.fa-bug:before {
-  content: "\f188";
-}
-.fa-vk:before {
-  content: "\f189";
-}
-.fa-weibo:before {
-  content: "\f18a";
-}
-.fa-renren:before {
-  content: "\f18b";
-}
-.fa-pagelines:before {
-  content: "\f18c";
-}
-.fa-stack-exchange:before {
-  content: "\f18d";
-}
-.fa-arrow-circle-o-right:before {
-  content: "\f18e";
-}
-.fa-arrow-circle-o-left:before {
-  content: "\f190";
-}
-.fa-toggle-left:before,
-.fa-caret-square-o-left:before {
-  content: "\f191";
-}
-.fa-dot-circle-o:before {
-  content: "\f192";
-}
-.fa-wheelchair:before {
-  content: "\f193";
-}
-.fa-vimeo-square:before {
-  content: "\f194";
-}
-.fa-turkish-lira:before,
-.fa-try:before {
-  content: "\f195";
-}
-.fa-plus-square-o:before {
-  content: "\f196";
-}
-.fa-space-shuttle:before {
-  content: "\f197";
-}
-.fa-slack:before {
-  content: "\f198";
-}
-.fa-envelope-square:before {
-  content: "\f199";
-}
-.fa-wordpress:before {
-  content: "\f19a";
-}
-.fa-openid:before {
-  content: "\f19b";
-}
-.fa-institution:before,
-.fa-bank:before,
-.fa-university:before {
-  content: "\f19c";
-}
-.fa-mortar-board:before,
-.fa-graduation-cap:before {
-  content: "\f19d";
-}
-.fa-yahoo:before {
-  content: "\f19e";
-}
-.fa-google:before {
-  content: "\f1a0";
-}
-.fa-reddit:before {
-  content: "\f1a1";
-}
-.fa-reddit-square:before {
-  content: "\f1a2";
-}
-.fa-stumbleupon-circle:before {
-  content: "\f1a3";
-}
-.fa-stumbleupon:before {
-  content: "\f1a4";
-}
-.fa-delicious:before {
-  content: "\f1a5";
-}
-.fa-digg:before {
-  content: "\f1a6";
-}
-.fa-pied-piper:before {
-  content: "\f1a7";
-}
-.fa-pied-piper-alt:before {
-  content: "\f1a8";
-}
-.fa-drupal:before {
-  content: "\f1a9";
-}
-.fa-joomla:before {
-  content: "\f1aa";
-}
-.fa-language:before {
-  content: "\f1ab";
-}
-.fa-fax:before {
-  content: "\f1ac";
-}
-.fa-building:before {
-  content: "\f1ad";
-}
-.fa-child:before {
-  content: "\f1ae";
-}
-.fa-paw:before {
-  content: "\f1b0";
-}
-.fa-spoon:before {
-  content: "\f1b1";
-}
-.fa-cube:before {
-  content: "\f1b2";
-}
-.fa-cubes:before {
-  content: "\f1b3";
-}
-.fa-behance:before {
-  content: "\f1b4";
-}
-.fa-behance-square:before {
-  content: "\f1b5";
-}
-.fa-steam:before {
-  content: "\f1b6";
-}
-.fa-steam-square:before {
-  content: "\f1b7";
-}
-.fa-recycle:before {
-  content: "\f1b8";
-}
-.fa-automobile:before,
-.fa-car:before {
-  content: "\f1b9";
-}
-.fa-cab:before,
-.fa-taxi:before {
-  content: "\f1ba";
-}
-.fa-tree:before {
-  content: "\f1bb";
-}
-.fa-spotify:before {
-  content: "\f1bc";
-}
-.fa-deviantart:before {
-  content: "\f1bd";
-}
-.fa-soundcloud:before {
-  content: "\f1be";
-}
-.fa-database:before {
-  content: "\f1c0";
-}
-.fa-file-pdf-o:before {
-  content: "\f1c1";
-}
-.fa-file-word-o:before {
-  content: "\f1c2";
-}
-.fa-file-excel-o:before {
-  content: "\f1c3";
-}
-.fa-file-powerpoint-o:before {
-  content: "\f1c4";
-}
-.fa-file-photo-o:before,
-.fa-file-picture-o:before,
-.fa-file-image-o:before {
-  content: "\f1c5";
-}
-.fa-file-zip-o:before,
-.fa-file-archive-o:before {
-  content: "\f1c6";
-}
-.fa-file-sound-o:before,
-.fa-file-audio-o:before {
-  content: "\f1c7";
-}
-.fa-file-movie-o:before,
-.fa-file-video-o:before {
-  content: "\f1c8";
-}
-.fa-file-code-o:before {
-  content: "\f1c9";
-}
-.fa-vine:before {
-  content: "\f1ca";
-}
-.fa-codepen:before {
-  content: "\f1cb";
-}
-.fa-jsfiddle:before {
-  content: "\f1cc";
-}
-.fa-life-bouy:before,
-.fa-life-buoy:before,
-.fa-life-saver:before,
-.fa-support:before,
-.fa-life-ring:before {
-  content: "\f1cd";
-}
-.fa-circle-o-notch:before {
-  content: "\f1ce";
-}
-.fa-ra:before,
-.fa-rebel:before {
-  content: "\f1d0";
-}
-.fa-ge:before,
-.fa-empire:before {
-  content: "\f1d1";
-}
-.fa-git-square:before {
-  content: "\f1d2";
-}
-.fa-git:before {
-  content: "\f1d3";
-}
-.fa-hacker-news:before {
-  content: "\f1d4";
-}
-.fa-tencent-weibo:before {
-  content: "\f1d5";
-}
-.fa-qq:before {
-  content: "\f1d6";
-}
-.fa-wechat:before,
-.fa-weixin:before {
-  content: "\f1d7";
-}
-.fa-send:before,
-.fa-paper-plane:before {
-  content: "\f1d8";
-}
-.fa-send-o:before,
-.fa-paper-plane-o:before {
-  content: "\f1d9";
-}
-.fa-history:before {
-  content: "\f1da";
-}
-.fa-genderless:before,
-.fa-circle-thin:before {
-  content: "\f1db";
-}
-.fa-header:before {
-  content: "\f1dc";
-}
-.fa-paragraph:before {
-  content: "\f1dd";
-}
-.fa-sliders:before {
-  content: "\f1de";
-}
-.fa-share-alt:before {
-  content: "\f1e0";
-}
-.fa-share-alt-square:before {
-  content: "\f1e1";
-}
-.fa-bomb:before {
-  content: "\f1e2";
-}
-.fa-soccer-ball-o:before,
-.fa-futbol-o:before {
-  content: "\f1e3";
-}
-.fa-tty:before {
-  content: "\f1e4";
-}
-.fa-binoculars:before {
-  content: "\f1e5";
-}
-.fa-plug:before {
-  content: "\f1e6";
-}
-.fa-slideshare:before {
-  content: "\f1e7";
-}
-.fa-twitch:before {
-  content: "\f1e8";
-}
-.fa-yelp:before {
-  content: "\f1e9";
-}
-.fa-newspaper-o:before {
-  content: "\f1ea";
-}
-.fa-wifi:before {
-  content: "\f1eb";
-}
-.fa-calculator:before {
-  content: "\f1ec";
-}
-.fa-paypal:before {
-  content: "\f1ed";
-}
-.fa-google-wallet:before {
-  content: "\f1ee";
-}
-.fa-cc-visa:before {
-  content: "\f1f0";
-}
-.fa-cc-mastercard:before {
-  content: "\f1f1";
-}
-.fa-cc-discover:before {
-  content: "\f1f2";
-}
-.fa-cc-amex:before {
-  content: "\f1f3";
-}
-.fa-cc-paypal:before {
-  content: "\f1f4";
-}
-.fa-cc-stripe:before {
-  content: "\f1f5";
-}
-.fa-bell-slash:before {
-  content: "\f1f6";
-}
-.fa-bell-slash-o:before {
-  content: "\f1f7";
-}
-.fa-trash:before {
-  content: "\f1f8";
-}
-.fa-copyright:before {
-  content: "\f1f9";
-}
-.fa-at:before {
-  content: "\f1fa";
-}
-.fa-eyedropper:before {
-  content: "\f1fb";
-}
-.fa-paint-brush:before {
-  content: "\f1fc";
-}
-.fa-birthday-cake:before {
-  content: "\f1fd";
-}
-.fa-area-chart:before {
-  content: "\f1fe";
-}
-.fa-pie-chart:before {
-  content: "\f200";
-}
-.fa-line-chart:before {
-  content: "\f201";
-}
-.fa-lastfm:before {
-  content: "\f202";
-}
-.fa-lastfm-square:before {
-  content: "\f203";
-}
-.fa-toggle-off:before {
-  content: "\f204";
-}
-.fa-toggle-on:before {
-  content: "\f205";
-}
-.fa-bicycle:before {
-  content: "\f206";
-}
-.fa-bus:before {
-  content: "\f207";
-}
-.fa-ioxhost:before {
-  content: "\f208";
-}
-.fa-angellist:before {
-  content: "\f209";
-}
-.fa-cc:before {
-  content: "\f20a";
-}
-.fa-shekel:before,
-.fa-sheqel:before,
-.fa-ils:before {
-  content: "\f20b";
-}
-.fa-meanpath:before {
-  content: "\f20c";
-}
-.fa-buysellads:before {
-  content: "\f20d";
-}
-.fa-connectdevelop:before {
-  content: "\f20e";
-}
-.fa-dashcube:before {
-  content: "\f210";
-}
-.fa-forumbee:before {
-  content: "\f211";
-}
-.fa-leanpub:before {
-  content: "\f212";
-}
-.fa-sellsy:before {
-  content: "\f213";
-}
-.fa-shirtsinbulk:before {
-  content: "\f214";
-}
-.fa-simplybuilt:before {
-  content: "\f215";
-}
-.fa-skyatlas:before {
-  content: "\f216";
-}
-.fa-cart-plus:before {
-  content: "\f217";
-}
-.fa-cart-arrow-down:before {
-  content: "\f218";
-}
-.fa-diamond:before {
-  content: "\f219";
-}
-.fa-ship:before {
-  content: "\f21a";
-}
-.fa-user-secret:before {
-  content: "\f21b";
-}
-.fa-motorcycle:before {
-  content: "\f21c";
-}
-.fa-street-view:before {
-  content: "\f21d";
-}
-.fa-heartbeat:before {
-  content: "\f21e";
-}
-.fa-venus:before {
-  content: "\f221";
-}
-.fa-mars:before {
-  content: "\f222";
-}
-.fa-mercury:before {
-  content: "\f223";
-}
-.fa-transgender:before {
-  content: "\f224";
-}
-.fa-transgender-alt:before {
-  content: "\f225";
-}
-.fa-venus-double:before {
-  content: "\f226";
-}
-.fa-mars-double:before {
-  content: "\f227";
-}
-.fa-venus-mars:before {
-  content: "\f228";
-}
-.fa-mars-stroke:before {
-  content: "\f229";
-}
-.fa-mars-stroke-v:before {
-  content: "\f22a";
-}
-.fa-mars-stroke-h:before {
-  content: "\f22b";
-}
-.fa-neuter:before {
-  content: "\f22c";
-}
-.fa-facebook-official:before {
-  content: "\f230";
-}
-.fa-pinterest-p:before {
-  content: "\f231";
-}
-.fa-whatsapp:before {
-  content: "\f232";
-}
-.fa-server:before {
-  content: "\f233";
-}
-.fa-user-plus:before {
-  content: "\f234";
-}
-.fa-user-times:before {
-  content: "\f235";
-}
-.fa-hotel:before,
-.fa-bed:before {
-  content: "\f236";
-}
-.fa-viacoin:before {
-  content: "\f237";
-}
-.fa-train:before {
-  content: "\f238";
-}
-.fa-subway:before {
-  content: "\f239";
-}
-.fa-medium:before {
-  content: "\f23a";
-}
-
-/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */
-html {
-  font-family: sans-serif;
-  -ms-text-size-adjust: 100%;
-  -webkit-text-size-adjust: 100%;
-}
-body {
-  margin: 0;
-}
-article,
-aside,
-details,
-figcaption,
-figure,
-footer,
-header,
-hgroup,
-main,
-menu,
-nav,
-section,
-summary {
-  display: block;
-}
-audio,
-canvas,
-progress,
-video {
-  display: inline-block;
-  vertical-align: baseline;
-}
-audio:not([controls]) {
-  display: none;
-  height: 0;
-}
-[hidden],
-template {
-  display: none;
-}
-a {
-  background-color: transparent;
-}
-a:active,
-a:hover {
-  outline: 0;
-}
-abbr[title] {
-  border-bottom: 1px dotted;
-}
-b,
-strong {
-  font-weight: bold;
-}
-dfn {
-  font-style: italic;
-}
-h1 {
-  font-size: 2em;
-  margin: 0.67em 0;
-}
-mark {
-  background: #ff0;
-  color: #000;
-}
-small {
-  font-size: 80%;
-}
-sub,
-sup {
-  font-size: 75%;
-  line-height: 0;
-  position: relative;
-  vertical-align: baseline;
-}
-sup {
-  top: -0.5em;
-}
-sub {
-  bottom: -0.25em;
-}
-img {
-  border: 0;
-}
-svg:not(:root) {
-  overflow: hidden;
-}
-figure {
-  margin: 1em 40px;
-}
-hr {
-  box-sizing: content-box;
-  height: 0;
-}
-pre {
-  overflow: auto;
-}
-code,
-kbd,
-pre,
-samp {
-  font-family: monospace, monospace;
-  font-size: 1em;
-}
-button,
-input,
-optgroup,
-select,
-textarea {
-  color: inherit;
-  font: inherit;
-  margin: 0;
-}
-button {
-  overflow: visible;
-}
-button,
-select {
-  text-transform: none;
-}
-button,
-html input[type="button"],
-input[type="reset"],
-input[type="submit"] {
-  -webkit-appearance: button;
-  cursor: pointer;
-}
-button[disabled],
-html input[disabled] {
-  cursor: default;
-}
-button::-moz-focus-inner,
-input::-moz-focus-inner {
-  border: 0;
-  padding: 0;
-}
-input {
-  line-height: normal;
-}
-input[type="checkbox"],
-input[type="radio"] {
-  box-sizing: border-box;
-  padding: 0;
-}
-input[type="number"]::-webkit-inner-spin-button,
-input[type="number"]::-webkit-outer-spin-button {
-  height: auto;
-}
-input[type="search"] {
-  -webkit-appearance: textfield;
-  box-sizing: content-box;
-}
-input[type="search"]::-webkit-search-cancel-button,
-input[type="search"]::-webkit-search-decoration {
-  -webkit-appearance: none;
-}
-fieldset {
-  border: 1px solid #c0c0c0;
-  margin: 0 2px;
-  padding: 0.35em 0.625em 0.75em;
-}
-legend {
-  border: 0;
-  padding: 0;
-}
-textarea {
-  overflow: auto;
-}
-optgroup {
-  font-weight: bold;
-}
-table {
-  border-collapse: collapse;
-  border-spacing: 0;
-}
-td,
-th {
-  padding: 0;
-}
-/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */
-@media print {
-  *,
-  *:before,
-  *:after {
-    background: transparent !important;
-    color: #000 !important;
-    box-shadow: none !important;
-    text-shadow: none !important;
-  }
-  a,
-  a:visited {
-    text-decoration: underline;
-  }
-  a[href]:after {
-    content: " (" attr(href) ")";
-  }
-  abbr[title]:after {
-    content: " (" attr(title) ")";
-  }
-  a[href^="#"]:after,
-  a[href^="javascript:"]:after {
-    content: "";
-  }
-  pre,
-  blockquote {
-    border: 1px solid #999;
-    page-break-inside: avoid;
-  }
-  thead {
-    display: table-header-group;
-  }
-  tr,
-  img {
-    page-break-inside: avoid;
-  }
-  img {
-    max-width: 100% !important;
-  }
-  p,
-  h2,
-  h3 {
-    orphans: 3;
-    widows: 3;
-  }
-  h2,
-  h3 {
-    page-break-after: avoid;
-  }
-  .navbar {
-    display: none;
-  }
-  .btn > .caret,
-  .dropup > .btn > .caret {
-    border-top-color: #000 !important;
-  }
-  .label {
-    border: 1px solid #000;
-  }
-  .table {
-    border-collapse: collapse !important;
-  }
-  .table td,
-  .table th {
-    background-color: #fff !important;
-  }
-  .table-bordered th,
-  .table-bordered td {
-    border: 1px solid #ddd !important;
-  }
-}
-@font-face {
-  font-family: 'Glyphicons Halflings';
-  src: url('../fonts/glyphicons-halflings-regular.eot');
-  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
-}
-.glyphicon {
-  position: relative;
-  top: 1px;
-  display: inline-block;
-  font-family: 'Glyphicons Halflings';
-  font-style: normal;
-  font-weight: normal;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-  -moz-osx-font-smoothing: grayscale;
-}
-.glyphicon-asterisk:before {
-  content: "\2a";
-}
-.glyphicon-plus:before {
-  content: "\2b";
-}
-.glyphicon-euro:before,
-.glyphicon-eur:before {
-  content: "\20ac";
-}
-.glyphicon-minus:before {
-  content: "\2212";
-}
-.glyphicon-cloud:before {
-  content: "\2601";
-}
-.glyphicon-envelope:before {
-  content: "\2709";
-}
-.glyphicon-pencil:before {
-  content: "\270f";
-}
-.glyphicon-glass:before {
-  content: "\e001";
-}
-.glyphicon-music:before {
-  content: "\e002";
-}
-.glyphicon-search:before {
-  content: "\e003";
-}
-.glyphicon-heart:before {
-  content: "\e005";
-}
-.glyphicon-star:before {
-  content: "\e006";
-}
-.glyphicon-star-empty:before {
-  content: "\e007";
-}
-.glyphicon-user:before {
-  content: "\e008";
-}
-.glyphicon-film:before {
-  content: "\e009";
-}
-.glyphicon-th-large:before {
-  content: "\e010";
-}
-.glyphicon-th:before {
-  content: "\e011";
-}
-.glyphicon-th-list:before {
-  content: "\e012";
-}
-.glyphicon-ok:before {
-  content: "\e013";
-}
-.glyphicon-remove:before {
-  content: "\e014";
-}
-.glyphicon-zoom-in:before {
-  content: "\e015";
-}
-.glyphicon-zoom-out:before {
-  content: "\e016";
-}
-.glyphicon-off:before {
-  content: "\e017";
-}
-.glyphicon-signal:before {
-  content: "\e018";
-}
-.glyphicon-cog:before {
-  content: "\e019";
-}
-.glyphicon-trash:before {
-  content: "\e020";
-}
-.glyphicon-home:before {
-  content: "\e021";
-}
-.glyphicon-file:before {
-  content: "\e022";
-}
-.glyphicon-time:before {
-  content: "\e023";
-}
-.glyphicon-road:before {
-  content: "\e024";
-}
-.glyphicon-download-alt:before {
-  content: "\e025";
-}
-.glyphicon-download:before {
-  content: "\e026";
-}
-.glyphicon-upload:before {
-  content: "\e027";
-}
-.glyphicon-inbox:before {
-  content: "\e028";
-}
-.glyphicon-play-circle:before {
-  content: "\e029";
-}
-.glyphicon-repeat:before {
-  content: "\e030";
-}
-.glyphicon-refresh:before {
-  content: "\e031";
-}
-.glyphicon-list-alt:before {
-  content: "\e032";
-}
-.glyphicon-lock:before {
-  content: "\e033";
-}
-.glyphicon-flag:before {
-  content: "\e034";
-}
-.glyphicon-headphones:before {
-  content: "\e035";
-}
-.glyphicon-volume-off:before {
-  content: "\e036";
-}
-.glyphicon-volume-down:before {
-  content: "\e037";
-}
-.glyphicon-volume-up:before {
-  content: "\e038";
-}
-.glyphicon-qrcode:before {
-  content: "\e039";
-}
-.glyphicon-barcode:before {
-  content: "\e040";
-}
-.glyphicon-tag:before {
-  content: "\e041";
-}
-.glyphicon-tags:before {
-  content: "\e042";
-}
-.glyphicon-book:before {
-  content: "\e043";
-}
-.glyphicon-bookmark:before {
-  content: "\e044";
-}
-.glyphicon-print:before {
-  content: "\e045";
-}
-.glyphicon-camera:before {
-  content: "\e046";
-}
-.glyphicon-font:before {
-  content: "\e047";
-}
-.glyphicon-bold:before {
-  content: "\e048";
-}
-.glyphicon-italic:before {
-  content: "\e049";
-}
-.glyphicon-text-height:before {
-  content: "\e050";
-}
-.glyphicon-text-width:before {
-  content: "\e051";
-}
-.glyphicon-align-left:before {
-  content: "\e052";
-}
-.glyphicon-align-center:before {
-  content: "\e053";
-}
-.glyphicon-align-right:before {
-  content: "\e054";
-}
-.glyphicon-align-justify:before {
-  content: "\e055";
-}
-.glyphicon-list:before {
-  content: "\e056";
-}
-.glyphicon-indent-left:before {
-  content: "\e057";
-}
-.glyphicon-indent-right:before {
-  content: "\e058";
-}
-.glyphicon-facetime-video:before {
-  content: "\e059";
-}
-.glyphicon-picture:before {
-  content: "\e060";
-}
-.glyphicon-map-marker:before {
-  content: "\e062";
-}
-.glyphicon-adjust:before {
-  content: "\e063";
-}
-.glyphicon-tint:before {
-  content: "\e064";
-}
-.glyphicon-edit:before {
-  content: "\e065";
-}
-.glyphicon-share:before {
-  content: "\e066";
-}
-.glyphicon-check:before {
-  content: "\e067";
-}
-.glyphicon-move:before {
-  content: "\e068";
-}
-.glyphicon-step-backward:before {
-  content: "\e069";
-}
-.glyphicon-fast-backward:before {
-  content: "\e070";
-}
-.glyphicon-backward:before {
-  content: "\e071";
-}
-.glyphicon-play:before {
-  content: "\e072";
-}
-.glyphicon-pause:before {
-  content: "\e073";
-}
-.glyphicon-stop:before {
-  content: "\e074";
-}
-.glyphicon-forward:before {
-  content: "\e075";
-}
-.glyphicon-fast-forward:before {
-  content: "\e076";
-}
-.glyphicon-step-forward:before {
-  content: "\e077";
-}
-.glyphicon-eject:before {
-  content: "\e078";
-}
-.glyphicon-chevron-left:before {
-  content: "\e079";
-}
-.glyphicon-chevron-right:before {
-  content: "\e080";
-}
-.glyphicon-plus-sign:before {
-  content: "\e081";
-}
-.glyphicon-minus-sign:before {
-  content: "\e082";
-}
-.glyphicon-remove-sign:before {
-  content: "\e083";
-}
-.glyphicon-ok-sign:before {
-  content: "\e084";
-}
-.glyphicon-question-sign:before {
-  content: "\e085";
-}
-.glyphicon-info-sign:before {
-  content: "\e086";
-}
-.glyphicon-screenshot:before {
-  content: "\e087";
-}
-.glyphicon-remove-circle:before {
-  content: "\e088";
-}
-.glyphicon-ok-circle:before {
-  content: "\e089";
-}
-.glyphicon-ban-circle:before {
-  content: "\e090";
-}
-.glyphicon-arrow-left:before {
-  content: "\e091";
-}
-.glyphicon-arrow-right:before {
-  content: "\e092";
-}
-.glyphicon-arrow-up:before {
-  content: "\e093";
-}
-.glyphicon-arrow-down:before {
-  content: "\e094";
-}
-.glyphicon-share-alt:before {
-  content: "\e095";
-}
-.glyphicon-resize-full:before {
-  content: "\e096";
-}
-.glyphicon-resize-small:before {
-  content: "\e097";
-}
-.glyphicon-exclamation-sign:before {
-  content: "\e101";
-}
-.glyphicon-gift:before {
-  content: "\e102";
-}
-.glyphicon-leaf:before {
-  content: "\e103";
-}
-.glyphicon-fire:before {
-  content: "\e104";
-}
-.glyphicon-eye-open:before {
-  content: "\e105";
-}
-.glyphicon-eye-close:before {
-  content: "\e106";
-}
-.glyphicon-warning-sign:before {
-  content: "\e107";
-}
-.glyphicon-plane:before {
-  content: "\e108";
-}
-.glyphicon-calendar:before {
-  content: "\e109";
-}
-.glyphicon-random:before {
-  content: "\e110";
-}
-.glyphicon-comment:before {
-  content: "\e111";
-}
-.glyphicon-magnet:before {
-  content: "\e112";
-}
-.glyphicon-chevron-up:before {
-  content: "\e113";
-}
-.glyphicon-chevron-down:before {
-  content: "\e114";
-}
-.glyphicon-retweet:before {
-  content: "\e115";
-}
-.glyphicon-shopping-cart:before {
-  content: "\e116";
-}
-.glyphicon-folder-close:before {
-  content: "\e117";
-}
-.glyphicon-folder-open:before {
-  content: "\e118";
-}
-.glyphicon-resize-vertical:before {
-  content: "\e119";
-}
-.glyphicon-resize-horizontal:before {
-  content: "\e120";
-}
-.glyphicon-hdd:before {
-  content: "\e121";
-}
-.glyphicon-bullhorn:before {
-  content: "\e122";
-}
-.glyphicon-bell:before {
-  content: "\e123";
-}
-.glyphicon-certificate:before {
-  content: "\e124";
-}
-.glyphicon-thumbs-up:before {
-  content: "\e125";
-}
-.glyphicon-thumbs-down:before {
-  content: "\e126";
-}
-.glyphicon-hand-right:before {
-  content: "\e127";
-}
-.glyphicon-hand-left:before {
-  content: "\e128";
-}
-.glyphicon-hand-up:before {
-  content: "\e129";
-}
-.glyphicon-hand-down:before {
-  content: "\e130";
-}
-.glyphicon-circle-arrow-right:before {
-  content: "\e131";
-}
-.glyphicon-circle-arrow-left:before {
-  content: "\e132";
-}
-.glyphicon-circle-arrow-up:before {
-  content: "\e133";
-}
-.glyphicon-circle-arrow-down:before {
-  content: "\e134";
-}
-.glyphicon-globe:before {
-  content: "\e135";
-}
-.glyphicon-wrench:before {
-  content: "\e136";
-}
-.glyphicon-tasks:before {
-  content: "\e137";
-}
-.glyphicon-filter:before {
-  content: "\e138";
-}
-.glyphicon-briefcase:before {
-  content: "\e139";
-}
-.glyphicon-fullscreen:before {
-  content: "\e140";
-}
-.glyphicon-dashboard:before {
-  content: "\e141";
-}
-.glyphicon-paperclip:before {
-  content: "\e142";
-}
-.glyphicon-heart-empty:before {
-  content: "\e143";
-}
-.glyphicon-link:before {
-  content: "\e144";
-}
-.glyphicon-phone:before {
-  content: "\e145";
-}
-.glyphicon-pushpin:before {
-  content: "\e146";
-}
-.glyphicon-usd:before {
-  content: "\e148";
-}
-.glyphicon-gbp:before {
-  content: "\e149";
-}
-.glyphicon-sort:before {
-  content: "\e150";
-}
-.glyphicon-sort-by-alphabet:before {
-  content: "\e151";
-}
-.glyphicon-sort-by-alphabet-alt:before {
-  content: "\e152";
-}
-.glyphicon-sort-by-order:before {
-  content: "\e153";
-}
-.glyphicon-sort-by-order-alt:before {
-  content: "\e154";
-}
-.glyphicon-sort-by-attributes:before {
-  content: "\e155";
-}
-.glyphicon-sort-by-attributes-alt:before {
-  content: "\e156";
-}
-.glyphicon-unchecked:before {
-  content: "\e157";
-}
-.glyphicon-expand:before {
-  content: "\e158";
-}
-.glyphicon-collapse-down:before {
-  content: "\e159";
-}
-.glyphicon-collapse-up:before {
-  content: "\e160";
-}
-.glyphicon-log-in:before {
-  content: "\e161";
-}
-.glyphicon-flash:before {
-  content: "\e162";
-}
-.glyphicon-log-out:before {
-  content: "\e163";
-}
-.glyphicon-new-window:before {
-  content: "\e164";
-}
-.glyphicon-record:before {
-  content: "\e165";
-}
-.glyphicon-save:before {
-  content: "\e166";
-}
-.glyphicon-open:before {
-  content: "\e167";
-}
-.glyphicon-saved:before {
-  content: "\e168";
-}
-.glyphicon-import:before {
-  content: "\e169";
-}
-.glyphicon-export:before {
-  content: "\e170";
-}
-.glyphicon-send:before {
-  content: "\e171";
-}
-.glyphicon-floppy-disk:before {
-  content: "\e172";
-}
-.glyphicon-floppy-saved:before {
-  content: "\e173";
-}
-.glyphicon-floppy-remove:before {
-  content: "\e174";
-}
-.glyphicon-floppy-save:before {
-  content: "\e175";
-}
-.glyphicon-floppy-open:before {
-  content: "\e176";
-}
-.glyphicon-credit-card:before {
-  content: "\e177";
-}
-.glyphicon-transfer:before {
-  content: "\e178";
-}
-.glyphicon-cutlery:before {
-  content: "\e179";
-}
-.glyphicon-header:before {
-  content: "\e180";
-}
-.glyphicon-compressed:before {
-  content: "\e181";
-}
-.glyphicon-earphone:before {
-  content: "\e182";
-}
-.glyphicon-phone-alt:before {
-  content: "\e183";
-}
-.glyphicon-tower:before {
-  content: "\e184";
-}
-.glyphicon-stats:before {
-  content: "\e185";
-}
-.glyphicon-sd-video:before {
-  content: "\e186";
-}
-.glyphicon-hd-video:before {
-  content: "\e187";
-}
-.glyphicon-subtitles:before {
-  content: "\e188";
-}
-.glyphicon-sound-stereo:before {
-  content: "\e189";
-}
-.glyphicon-sound-dolby:before {
-  content: "\e190";
-}
-.glyphicon-sound-5-1:before {
-  content: "\e191";
-}
-.glyphicon-sound-6-1:before {
-  content: "\e192";
-}
-.glyphicon-sound-7-1:before {
-  content: "\e193";
-}
-.glyphicon-copyright-mark:before {
-  content: "\e194";
-}
-.glyphicon-registration-mark:before {
-  content: "\e195";
-}
-.glyphicon-cloud-download:before {
-  content: "\e197";
-}
-.glyphicon-cloud-upload:before {
-  content: "\e198";
-}
-.glyphicon-tree-conifer:before {
-  content: "\e199";
-}
-.glyphicon-tree-deciduous:before {
-  content: "\e200";
-}
-.glyphicon-cd:before {
-  content: "\e201";
-}
-.glyphicon-save-file:before {
-  content: "\e202";
-}
-.glyphicon-open-file:before {
-  content: "\e203";
-}
-.glyphicon-level-up:before {
-  content: "\e204";
-}
-.glyphicon-copy:before {
-  content: "\e205";
-}
-.glyphicon-paste:before {
-  content: "\e206";
-}
-.glyphicon-alert:before {
-  content: "\e209";
-}
-.glyphicon-equalizer:before {
-  content: "\e210";
-}
-.glyphicon-king:before {
-  content: "\e211";
-}
-.glyphicon-queen:before {
-  content: "\e212";
-}
-.glyphicon-pawn:before {
-  content: "\e213";
-}
-.glyphicon-bishop:before {
-  content: "\e214";
-}
-.glyphicon-knight:before {
-  content: "\e215";
-}
-.glyphicon-baby-formula:before {
-  content: "\e216";
-}
-.glyphicon-tent:before {
-  content: "\26fa";
-}
-.glyphicon-blackboard:before {
-  content: "\e218";
-}
-.glyphicon-bed:before {
-  content: "\e219";
-}
-.glyphicon-apple:before {
-  content: "\f8ff";
-}
-.glyphicon-erase:before {
-  content: "\e221";
-}
-.glyphicon-hourglass:before {
-  content: "\231b";
-}
-.glyphicon-lamp:before {
-  content: "\e223";
-}
-.glyphicon-duplicate:before {
-  content: "\e224";
-}
-.glyphicon-piggy-bank:before {
-  content: "\e225";
-}
-.glyphicon-scissors:before {
-  content: "\e226";
-}
-.glyphicon-bitcoin:before {
-  content: "\e227";
-}
-.glyphicon-btc:before {
-  content: "\e227";
-}
-.glyphicon-xbt:before {
-  content: "\e227";
-}
-.glyphicon-yen:before {
-  content: "\00a5";
-}
-.glyphicon-jpy:before {
-  content: "\00a5";
-}
-.glyphicon-ruble:before {
-  content: "\20bd";
-}
-.glyphicon-rub:before {
-  content: "\20bd";
-}
-.glyphicon-scale:before {
-  content: "\e230";
-}
-.glyphicon-ice-lolly:before {
-  content: "\e231";
-}
-.glyphicon-ice-lolly-tasted:before {
-  content: "\e232";
-}
-.glyphicon-education:before {
-  content: "\e233";
-}
-.glyphicon-option-horizontal:before {
-  content: "\e234";
-}
-.glyphicon-option-vertical:before {
-  content: "\e235";
-}
-.glyphicon-menu-hamburger:before {
-  content: "\e236";
-}
-.glyphicon-modal-window:before {
-  content: "\e237";
-}
-.glyphicon-oil:before {
-  content: "\e238";
-}
-.glyphicon-grain:before {
-  content: "\e239";
-}
-.glyphicon-sunglasses:before {
-  content: "\e240";
-}
-.glyphicon-text-size:before {
-  content: "\e241";
-}
-.glyphicon-text-color:before {
-  content: "\e242";
-}
-.glyphicon-text-background:before {
-  content: "\e243";
-}
-.glyphicon-object-align-top:before {
-  content: "\e244";
-}
-.glyphicon-object-align-bottom:before {
-  content: "\e245";
-}
-.glyphicon-object-align-horizontal:before {
-  content: "\e246";
-}
-.glyphicon-object-align-left:before {
-  content: "\e247";
-}
-.glyphicon-object-align-vertical:before {
-  content: "\e248";
-}
-.glyphicon-object-align-right:before {
-  content: "\e249";
-}
-.glyphicon-triangle-right:before {
-  content: "\e250";
-}
-.glyphicon-triangle-left:before {
-  content: "\e251";
-}
-.glyphicon-triangle-bottom:before {
-  content: "\e252";
-}
-.glyphicon-triangle-top:before {
-  content: "\e253";
-}
-.glyphicon-console:before {
-  content: "\e254";
-}
-.glyphicon-superscript:before {
-  content: "\e255";
-}
-.glyphicon-subscript:before {
-  content: "\e256";
-}
-.glyphicon-menu-left:before {
-  content: "\e257";
-}
-.glyphicon-menu-right:before {
-  content: "\e258";
-}
-.glyphicon-menu-down:before {
-  content: "\e259";
-}
-.glyphicon-menu-up:before {
-  content: "\e260";
-}
-* {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-*:before,
-*:after {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-html {
-  font-size: 10px;
-  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
-}
-body {
-  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #333333;
-  background-color: #fff;
-}
-input,
-button,
-select,
-textarea {
-  font-family: inherit;
-  font-size: inherit;
-  line-height: inherit;
-}
-a {
-  color: #158cba;
-  text-decoration: none;
-}
-a:hover,
-a:focus {
-  color: #0d5875;
-  text-decoration: underline;
-}
-a:focus {
-  outline: thin dotted;
-  outline: 5px auto -webkit-focus-ring-color;
-  outline-offset: -2px;
-}
-figure {
-  margin: 0;
-}
-img {
-  vertical-align: middle;
-}
-.img-responsive,
-.thumbnail > img,
-.thumbnail a > img,
-.carousel-inner > .item > img,
-.carousel-inner > .item > a > img {
-  display: block;
-  max-width: 100%;
-  height: auto;
-}
-.img-rounded {
-  border-radius: 6px;
-}
-.img-thumbnail {
-  padding: 4px;
-  line-height: 1.42857143;
-  background-color: #fff;
-  border: 1px solid #ddd;
-  border-radius: 4px;
-  -webkit-transition: all 0.2s ease-in-out;
-  -o-transition: all 0.2s ease-in-out;
-  transition: all 0.2s ease-in-out;
-  display: inline-block;
-  max-width: 100%;
-  height: auto;
-}
-.img-circle {
-  border-radius: 50%;
-}
-hr {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  border: 0;
-  border-top: 1px solid #eeeeee;
-}
-.sr-only {
-  position: absolute;
-  width: 1px;
-  height: 1px;
-  margin: -1px;
-  padding: 0;
-  overflow: hidden;
-  clip: rect(0, 0, 0, 0);
-  border: 0;
-}
-.sr-only-focusable:active,
-.sr-only-focusable:focus {
-  position: static;
-  width: auto;
-  height: auto;
-  margin: 0;
-  overflow: visible;
-  clip: auto;
-}
-[role="button"] {
-  cursor: pointer;
-}
-h1,
-h2,
-h3,
-h4,
-h5,
-h6,
-.h1,
-.h2,
-.h3,
-.h4,
-.h5,
-.h6 {
-  font-family: inherit;
-  font-weight: 500;
-  line-height: 1.1;
-  color: inherit;
-}
-h1 small,
-h2 small,
-h3 small,
-h4 small,
-h5 small,
-h6 small,
-.h1 small,
-.h2 small,
-.h3 small,
-.h4 small,
-.h5 small,
-.h6 small,
-h1 .small,
-h2 .small,
-h3 .small,
-h4 .small,
-h5 .small,
-h6 .small,
-.h1 .small,
-.h2 .small,
-.h3 .small,
-.h4 .small,
-.h5 .small,
-.h6 .small {
-  font-weight: normal;
-  line-height: 1;
-  color: #777777;
-}
-h1,
-.h1,
-h2,
-.h2,
-h3,
-.h3 {
-  margin-top: 20px;
-  margin-bottom: 10px;
-}
-h1 small,
-.h1 small,
-h2 small,
-.h2 small,
-h3 small,
-.h3 small,
-h1 .small,
-.h1 .small,
-h2 .small,
-.h2 .small,
-h3 .small,
-.h3 .small {
-  font-size: 65%;
-}
-h4,
-.h4,
-h5,
-.h5,
-h6,
-.h6 {
-  margin-top: 10px;
-  margin-bottom: 10px;
-}
-h4 small,
-.h4 small,
-h5 small,
-.h5 small,
-h6 small,
-.h6 small,
-h4 .small,
-.h4 .small,
-h5 .small,
-.h5 .small,
-h6 .small,
-.h6 .small {
-  font-size: 75%;
-}
-h1,
-.h1 {
-  font-size: 28px;
-}
-h2,
-.h2 {
-  font-size: 23px;
-}
-h3,
-.h3 {
-  font-size: 21px;
-}
-h4,
-.h4 {
-  font-size: 18px;
-}
-h5,
-.h5 {
-  font-size: 14px;
-}
-h6,
-.h6 {
-  font-size: 12px;
-}
-p {
-  margin: 0 0 10px;
-}
-.lead {
-  margin-bottom: 20px;
-  font-size: 16px;
-  font-weight: 300;
-  line-height: 1.4;
-}
-@media (min-width: 768px) {
-  .lead {
-    font-size: 21px;
-  }
-}
-small,
-.small {
-  font-size: 85%;
-}
-mark,
-.mark {
-  background-color: #fcf8e3;
-  padding: .2em;
-}
-.text-left {
-  text-align: left;
-}
-.text-right {
-  text-align: right;
-}
-.text-center {
-  text-align: center;
-}
-.text-justify {
-  text-align: justify;
-}
-.text-nowrap {
-  white-space: nowrap;
-}
-.text-lowercase {
-  text-transform: lowercase;
-}
-.text-uppercase {
-  text-transform: uppercase;
-}
-.text-capitalize {
-  text-transform: capitalize;
-}
-.text-muted {
-  color: #777777;
-}
-.text-primary {
-  color: #158cba;
-}
-a.text-primary:hover,
-a.text-primary:focus {
-  color: #106a8c;
-}
-.text-success {
-  color: #3c763d;
-}
-a.text-success:hover,
-a.text-success:focus {
-  color: #2b542c;
-}
-.text-info {
-  color: #31708f;
-}
-a.text-info:hover,
-a.text-info:focus {
-  color: #245269;
-}
-.text-warning {
-  color: #8a6d3b;
-}
-a.text-warning:hover,
-a.text-warning:focus {
-  color: #66512c;
-}
-.text-danger {
-  color: #a94442;
-}
-a.text-danger:hover,
-a.text-danger:focus {
-  color: #843534;
-}
-.bg-primary {
-  color: #fff;
-  background-color: #158cba;
-}
-a.bg-primary:hover,
-a.bg-primary:focus {
-  background-color: #106a8c;
-}
-.bg-success {
-  background-color: #dff0d8;
-}
-a.bg-success:hover,
-a.bg-success:focus {
-  background-color: #c1e2b3;
-}
-.bg-info {
-  background-color: #d9edf7;
-}
-a.bg-info:hover,
-a.bg-info:focus {
-  background-color: #afd9ee;
-}
-.bg-warning {
-  background-color: #fcf8e3;
-}
-a.bg-warning:hover,
-a.bg-warning:focus {
-  background-color: #f7ecb5;
-}
-.bg-danger {
-  background-color: #f2dede;
-}
-a.bg-danger:hover,
-a.bg-danger:focus {
-  background-color: #e4b9b9;
-}
-.page-header {
-  padding-bottom: 9px;
-  margin: 40px 0 20px;
-  border-bottom: 1px solid #eeeeee;
-}
-ul,
-ol {
-  margin-top: 0;
-  margin-bottom: 10px;
-}
-ul ul,
-ol ul,
-ul ol,
-ol ol {
-  margin-bottom: 0;
-}
-.list-unstyled {
-  padding-left: 0;
-  list-style: none;
-}
-.list-inline {
-  padding-left: 0;
-  list-style: none;
-  margin-left: -5px;
-}
-.list-inline > li {
-  display: inline-block;
-  padding-left: 5px;
-  padding-right: 5px;
-}
-dl {
-  margin-top: 0;
-  margin-bottom: 20px;
-}
-dt,
-dd {
-  line-height: 1.42857143;
-}
-dt {
-  font-weight: bold;
-}
-dd {
-  margin-left: 0;
-}
-@media (min-width: 768px) {
-  .dl-horizontal dt {
-    float: left;
-    width: 160px;
-    clear: left;
-    text-align: right;
-    overflow: hidden;
-    text-overflow: ellipsis;
-    white-space: nowrap;
-  }
-  .dl-horizontal dd {
-    margin-left: 180px;
-  }
-}
-abbr[title],
-abbr[data-original-title] {
-  cursor: help;
-  border-bottom: 1px dotted #777777;
-}
-.initialism {
-  font-size: 90%;
-  text-transform: uppercase;
-}
-blockquote {
-  padding: 10px 20px;
-  margin: 0 0 20px;
-  font-size: 17.5px;
-  border-left: 5px solid #eeeeee;
-}
-blockquote p:last-child,
-blockquote ul:last-child,
-blockquote ol:last-child {
-  margin-bottom: 0;
-}
-blockquote footer,
-blockquote small,
-blockquote .small {
-  display: block;
-  font-size: 80%;
-  line-height: 1.42857143;
-  color: #777777;
-}
-blockquote footer:before,
-blockquote small:before,
-blockquote .small:before {
-  content: '\2014 \00A0';
-}
-.blockquote-reverse,
-blockquote.pull-right {
-  padding-right: 15px;
-  padding-left: 0;
-  border-right: 5px solid #eeeeee;
-  border-left: 0;
-  text-align: right;
-}
-.blockquote-reverse footer:before,
-blockquote.pull-right footer:before,
-.blockquote-reverse small:before,
-blockquote.pull-right small:before,
-.blockquote-reverse .small:before,
-blockquote.pull-right .small:before {
-  content: '';
-}
-.blockquote-reverse footer:after,
-blockquote.pull-right footer:after,
-.blockquote-reverse small:after,
-blockquote.pull-right small:after,
-.blockquote-reverse .small:after,
-blockquote.pull-right .small:after {
-  content: '\00A0 \2014';
-}
-address {
-  margin-bottom: 20px;
-  font-style: normal;
-  line-height: 1.42857143;
-}
-code,
-kbd,
-pre,
-samp {
-  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
-}
-code {
-  padding: 2px 4px;
-  font-size: 90%;
-  color: #c7254e;
-  background-color: #f9f2f4;
-  border-radius: 4px;
-}
-kbd {
-  padding: 2px 4px;
-  font-size: 90%;
-  color: #fff;
-  background-color: #333;
-  border-radius: 3px;
-  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);
-}
-kbd kbd {
-  padding: 0;
-  font-size: 100%;
-  font-weight: bold;
-  box-shadow: none;
-}
-pre {
-  display: block;
-  padding: 9.5px;
-  margin: 0 0 10px;
-  font-size: 13px;
-  line-height: 1.42857143;
-  word-break: break-all;
-  word-wrap: break-word;
-  color: #333333;
-  background-color: #f5f5f5;
-  border: 1px solid #ccc;
-  border-radius: 4px;
-}
-pre code {
-  padding: 0;
-  font-size: inherit;
-  color: inherit;
-  white-space: pre-wrap;
-  background-color: transparent;
-  border-radius: 0;
-}
-.pre-scrollable {
-  max-height: 340px;
-  overflow-y: scroll;
-}
-.container {
-  margin-right: auto;
-  margin-left: auto;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-@media (min-width: 768px) {
-  .container {
-    width: 750px;
-  }
-}
-@media (min-width: 992px) {
-  .container {
-    width: 970px;
-  }
-}
-@media (min-width: 1200px) {
-  .container {
-    width: 1170px;
-  }
-}
-.container-fluid {
-  margin-right: auto;
-  margin-left: auto;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-.row {
-  margin-left: -15px;
-  margin-right: -15px;
-}
-.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
-  position: relative;
-  min-height: 1px;
-  padding-left: 15px;
-  padding-right: 15px;
-}
-.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
-  float: left;
-}
-.col-xs-12 {
-  width: 100%;
-}
-.col-xs-11 {
-  width: 91.66666667%;
-}
-.col-xs-10 {
-  width: 83.33333333%;
-}
-.col-xs-9 {
-  width: 75%;
-}
-.col-xs-8 {
-  width: 66.66666667%;
-}
-.col-xs-7 {
-  width: 58.33333333%;
-}
-.col-xs-6 {
-  width: 50%;
-}
-.col-xs-5 {
-  width: 41.66666667%;
-}
-.col-xs-4 {
-  width: 33.33333333%;
-}
-.col-xs-3 {
-  width: 25%;
-}
-.col-xs-2 {
-  width: 16.66666667%;
-}
-.col-xs-1 {
-  width: 8.33333333%;
-}
-.col-xs-pull-12 {
-  right: 100%;
-}
-.col-xs-pull-11 {
-  right: 91.66666667%;
-}
-.col-xs-pull-10 {
-  right: 83.33333333%;
-}
-.col-xs-pull-9 {
-  right: 75%;
-}
-.col-xs-pull-8 {
-  right: 66.66666667%;
-}
-.col-xs-pull-7 {
-  right: 58.33333333%;
-}
-.col-xs-pull-6 {
-  right: 50%;
-}
-.col-xs-pull-5 {
-  right: 41.66666667%;
-}
-.col-xs-pull-4 {
-  right: 33.33333333%;
-}
-.col-xs-pull-3 {
-  right: 25%;
-}
-.col-xs-pull-2 {
-  right: 16.66666667%;
-}
-.col-xs-pull-1 {
-  right: 8.33333333%;
-}
-.col-xs-pull-0 {
-  right: auto;
-}
-.col-xs-push-12 {
-  left: 100%;
-}
-.col-xs-push-11 {
-  left: 91.66666667%;
-}
-.col-xs-push-10 {
-  left: 83.33333333%;
-}
-.col-xs-push-9 {
-  left: 75%;
-}
-.col-xs-push-8 {
-  left: 66.66666667%;
-}
-.col-xs-push-7 {
-  left: 58.33333333%;
-}
-.col-xs-push-6 {
-  left: 50%;
-}
-.col-xs-push-5 {
-  left: 41.66666667%;
-}
-.col-xs-push-4 {
-  left: 33.33333333%;
-}
-.col-xs-push-3 {
-  left: 25%;
-}
-.col-xs-push-2 {
-  left: 16.66666667%;
-}
-.col-xs-push-1 {
-  left: 8.33333333%;
-}
-.col-xs-push-0 {
-  left: auto;
-}
-.col-xs-offset-12 {
-  margin-left: 100%;
-}
-.col-xs-offset-11 {
-  margin-left: 91.66666667%;
-}
-.col-xs-offset-10 {
-  margin-left: 83.33333333%;
-}
-.col-xs-offset-9 {
-  margin-left: 75%;
-}
-.col-xs-offset-8 {
-  margin-left: 66.66666667%;
-}
-.col-xs-offset-7 {
-  margin-left: 58.33333333%;
-}
-.col-xs-offset-6 {
-  margin-left: 50%;
-}
-.col-xs-offset-5 {
-  margin-left: 41.66666667%;
-}
-.col-xs-offset-4 {
-  margin-left: 33.33333333%;
-}
-.col-xs-offset-3 {
-  margin-left: 25%;
-}
-.col-xs-offset-2 {
-  margin-left: 16.66666667%;
-}
-.col-xs-offset-1 {
-  margin-left: 8.33333333%;
-}
-.col-xs-offset-0 {
-  margin-left: 0%;
-}
-@media (min-width: 768px) {
-  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
-    float: left;
-  }
-  .col-sm-12 {
-    width: 100%;
-  }
-  .col-sm-11 {
-    width: 91.66666667%;
-  }
-  .col-sm-10 {
-    width: 83.33333333%;
-  }
-  .col-sm-9 {
-    width: 75%;
-  }
-  .col-sm-8 {
-    width: 66.66666667%;
-  }
-  .col-sm-7 {
-    width: 58.33333333%;
-  }
-  .col-sm-6 {
-    width: 50%;
-  }
-  .col-sm-5 {
-    width: 41.66666667%;
-  }
-  .col-sm-4 {
-    width: 33.33333333%;
-  }
-  .col-sm-3 {
-    width: 25%;
-  }
-  .col-sm-2 {
-    width: 16.66666667%;
-  }
-  .col-sm-1 {
-    width: 8.33333333%;
-  }
-  .col-sm-pull-12 {
-    right: 100%;
-  }
-  .col-sm-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-sm-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-sm-pull-9 {
-    right: 75%;
-  }
-  .col-sm-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-sm-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-sm-pull-6 {
-    right: 50%;
-  }
-  .col-sm-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-sm-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-sm-pull-3 {
-    right: 25%;
-  }
-  .col-sm-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-sm-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-sm-pull-0 {
-    right: auto;
-  }
-  .col-sm-push-12 {
-    left: 100%;
-  }
-  .col-sm-push-11 {
-    left: 91.66666667%;
-  }
-  .col-sm-push-10 {
-    left: 83.33333333%;
-  }
-  .col-sm-push-9 {
-    left: 75%;
-  }
-  .col-sm-push-8 {
-    left: 66.66666667%;
-  }
-  .col-sm-push-7 {
-    left: 58.33333333%;
-  }
-  .col-sm-push-6 {
-    left: 50%;
-  }
-  .col-sm-push-5 {
-    left: 41.66666667%;
-  }
-  .col-sm-push-4 {
-    left: 33.33333333%;
-  }
-  .col-sm-push-3 {
-    left: 25%;
-  }
-  .col-sm-push-2 {
-    left: 16.66666667%;
-  }
-  .col-sm-push-1 {
-    left: 8.33333333%;
-  }
-  .col-sm-push-0 {
-    left: auto;
-  }
-  .col-sm-offset-12 {
-    margin-left: 100%;
-  }
-  .col-sm-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-sm-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-sm-offset-9 {
-    margin-left: 75%;
-  }
-  .col-sm-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-sm-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-sm-offset-6 {
-    margin-left: 50%;
-  }
-  .col-sm-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-sm-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-sm-offset-3 {
-    margin-left: 25%;
-  }
-  .col-sm-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-sm-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-sm-offset-0 {
-    margin-left: 0%;
-  }
-}
-@media (min-width: 992px) {
-  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
-    float: left;
-  }
-  .col-md-12 {
-    width: 100%;
-  }
-  .col-md-11 {
-    width: 91.66666667%;
-  }
-  .col-md-10 {
-    width: 83.33333333%;
-  }
-  .col-md-9 {
-    width: 75%;
-  }
-  .col-md-8 {
-    width: 66.66666667%;
-  }
-  .col-md-7 {
-    width: 58.33333333%;
-  }
-  .col-md-6 {
-    width: 50%;
-  }
-  .col-md-5 {
-    width: 41.66666667%;
-  }
-  .col-md-4 {
-    width: 33.33333333%;
-  }
-  .col-md-3 {
-    width: 25%;
-  }
-  .col-md-2 {
-    width: 16.66666667%;
-  }
-  .col-md-1 {
-    width: 8.33333333%;
-  }
-  .col-md-pull-12 {
-    right: 100%;
-  }
-  .col-md-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-md-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-md-pull-9 {
-    right: 75%;
-  }
-  .col-md-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-md-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-md-pull-6 {
-    right: 50%;
-  }
-  .col-md-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-md-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-md-pull-3 {
-    right: 25%;
-  }
-  .col-md-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-md-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-md-pull-0 {
-    right: auto;
-  }
-  .col-md-push-12 {
-    left: 100%;
-  }
-  .col-md-push-11 {
-    left: 91.66666667%;
-  }
-  .col-md-push-10 {
-    left: 83.33333333%;
-  }
-  .col-md-push-9 {
-    left: 75%;
-  }
-  .col-md-push-8 {
-    left: 66.66666667%;
-  }
-  .col-md-push-7 {
-    left: 58.33333333%;
-  }
-  .col-md-push-6 {
-    left: 50%;
-  }
-  .col-md-push-5 {
-    left: 41.66666667%;
-  }
-  .col-md-push-4 {
-    left: 33.33333333%;
-  }
-  .col-md-push-3 {
-    left: 25%;
-  }
-  .col-md-push-2 {
-    left: 16.66666667%;
-  }
-  .col-md-push-1 {
-    left: 8.33333333%;
-  }
-  .col-md-push-0 {
-    left: auto;
-  }
-  .col-md-offset-12 {
-    margin-left: 100%;
-  }
-  .col-md-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-md-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-md-offset-9 {
-    margin-left: 75%;
-  }
-  .col-md-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-md-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-md-offset-6 {
-    margin-left: 50%;
-  }
-  .col-md-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-md-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-md-offset-3 {
-    margin-left: 25%;
-  }
-  .col-md-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-md-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-md-offset-0 {
-    margin-left: 0%;
-  }
-}
-@media (min-width: 1200px) {
-  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
-    float: left;
-  }
-  .col-lg-12 {
-    width: 100%;
-  }
-  .col-lg-11 {
-    width: 91.66666667%;
-  }
-  .col-lg-10 {
-    width: 83.33333333%;
-  }
-  .col-lg-9 {
-    width: 75%;
-  }
-  .col-lg-8 {
-    width: 66.66666667%;
-  }
-  .col-lg-7 {
-    width: 58.33333333%;
-  }
-  .col-lg-6 {
-    width: 50%;
-  }
-  .col-lg-5 {
-    width: 41.66666667%;
-  }
-  .col-lg-4 {
-    width: 33.33333333%;
-  }
-  .col-lg-3 {
-    width: 25%;
-  }
-  .col-lg-2 {
-    width: 16.66666667%;
-  }
-  .col-lg-1 {
-    width: 8.33333333%;
-  }
-  .col-lg-pull-12 {
-    right: 100%;
-  }
-  .col-lg-pull-11 {
-    right: 91.66666667%;
-  }
-  .col-lg-pull-10 {
-    right: 83.33333333%;
-  }
-  .col-lg-pull-9 {
-    right: 75%;
-  }
-  .col-lg-pull-8 {
-    right: 66.66666667%;
-  }
-  .col-lg-pull-7 {
-    right: 58.33333333%;
-  }
-  .col-lg-pull-6 {
-    right: 50%;
-  }
-  .col-lg-pull-5 {
-    right: 41.66666667%;
-  }
-  .col-lg-pull-4 {
-    right: 33.33333333%;
-  }
-  .col-lg-pull-3 {
-    right: 25%;
-  }
-  .col-lg-pull-2 {
-    right: 16.66666667%;
-  }
-  .col-lg-pull-1 {
-    right: 8.33333333%;
-  }
-  .col-lg-pull-0 {
-    right: auto;
-  }
-  .col-lg-push-12 {
-    left: 100%;
-  }
-  .col-lg-push-11 {
-    left: 91.66666667%;
-  }
-  .col-lg-push-10 {
-    left: 83.33333333%;
-  }
-  .col-lg-push-9 {
-    left: 75%;
-  }
-  .col-lg-push-8 {
-    left: 66.66666667%;
-  }
-  .col-lg-push-7 {
-    left: 58.33333333%;
-  }
-  .col-lg-push-6 {
-    left: 50%;
-  }
-  .col-lg-push-5 {
-    left: 41.66666667%;
-  }
-  .col-lg-push-4 {
-    left: 33.33333333%;
-  }
-  .col-lg-push-3 {
-    left: 25%;
-  }
-  .col-lg-push-2 {
-    left: 16.66666667%;
-  }
-  .col-lg-push-1 {
-    left: 8.33333333%;
-  }
-  .col-lg-push-0 {
-    left: auto;
-  }
-  .col-lg-offset-12 {
-    margin-left: 100%;
-  }
-  .col-lg-offset-11 {
-    margin-left: 91.66666667%;
-  }
-  .col-lg-offset-10 {
-    margin-left: 83.33333333%;
-  }
-  .col-lg-offset-9 {
-    margin-left: 75%;
-  }
-  .col-lg-offset-8 {
-    margin-left: 66.66666667%;
-  }
-  .col-lg-offset-7 {
-    margin-left: 58.33333333%;
-  }
-  .col-lg-offset-6 {
-    margin-left: 50%;
-  }
-  .col-lg-offset-5 {
-    margin-left: 41.66666667%;
-  }
-  .col-lg-offset-4 {
-    margin-left: 33.33333333%;
-  }
-  .col-lg-offset-3 {
-    margin-left: 25%;
-  }
-  .col-lg-offset-2 {
-    margin-left: 16.66666667%;
-  }
-  .col-lg-offset-1 {
-    margin-left: 8.33333333%;
-  }
-  .col-lg-offset-0 {
-    margin-left: 0%;
-  }
-}
-table {
-  background-color: transparent;
-}
-caption {
-  padding-top: 8px;
-  padding-bottom: 8px;
-  color: #777777;
-  text-align: left;
-}
-th {
-  text-align: left;
-}
-.table {
-  width: 100%;
-  max-width: 100%;
-  margin-bottom: 20px;
-}
-.table > thead > tr > th,
-.table > tbody > tr > th,
-.table > tfoot > tr > th,
-.table > thead > tr > td,
-.table > tbody > tr > td,
-.table > tfoot > tr > td {
-  padding: 8px;
-  line-height: 1.42857143;
-  vertical-align: top;
-  border-top: 1px solid #ddd;
-}
-.table > thead > tr > th {
-  vertical-align: bottom;
-  border-bottom: 2px solid #ddd;
-}
-.table > caption + thead > tr:first-child > th,
-.table > colgroup + thead > tr:first-child > th,
-.table > thead:first-child > tr:first-child > th,
-.table > caption + thead > tr:first-child > td,
-.table > colgroup + thead > tr:first-child > td,
-.table > thead:first-child > tr:first-child > td {
-  border-top: 0;
-}
-.table > tbody + tbody {
-  border-top: 2px solid #ddd;
-}
-.table .table {
-  background-color: #fff;
-}
-.table-condensed > thead > tr > th,
-.table-condensed > tbody > tr > th,
-.table-condensed > tfoot > tr > th,
-.table-condensed > thead > tr > td,
-.table-condensed > tbody > tr > td,
-.table-condensed > tfoot > tr > td {
-  padding: 5px;
-}
-.table-bordered {
-  border: 1px solid #ddd;
-}
-.table-bordered > thead > tr > th,
-.table-bordered > tbody > tr > th,
-.table-bordered > tfoot > tr > th,
-.table-bordered > thead > tr > td,
-.table-bordered > tbody > tr > td,
-.table-bordered > tfoot > tr > td {
-  border: 1px solid #ddd;
-}
-.table-bordered > thead > tr > th,
-.table-bordered > thead > tr > td {
-  border-bottom-width: 2px;
-}
-.table-striped > tbody > tr:nth-of-type(odd) {
-  background-color: #f9f9f9;
-}
-.table-hover > tbody > tr:hover {
-  background-color: #f5f5f5;
-}
-table col[class*="col-"] {
-  position: static;
-  float: none;
-  display: table-column;
-}
-table td[class*="col-"],
-table th[class*="col-"] {
-  position: static;
-  float: none;
-  display: table-cell;
-}
-.table > thead > tr > td.active,
-.table > tbody > tr > td.active,
-.table > tfoot > tr > td.active,
-.table > thead > tr > th.active,
-.table > tbody > tr > th.active,
-.table > tfoot > tr > th.active,
-.table > thead > tr.active > td,
-.table > tbody > tr.active > td,
-.table > tfoot > tr.active > td,
-.table > thead > tr.active > th,
-.table > tbody > tr.active > th,
-.table > tfoot > tr.active > th {
-  background-color: #f5f5f5;
-}
-.table-hover > tbody > tr > td.active:hover,
-.table-hover > tbody > tr > th.active:hover,
-.table-hover > tbody > tr.active:hover > td,
-.table-hover > tbody > tr:hover > .active,
-.table-hover > tbody > tr.active:hover > th {
-  background-color: #e8e8e8;
-}
-.table > thead > tr > td.success,
-.table > tbody > tr > td.success,
-.table > tfoot > tr > td.success,
-.table > thead > tr > th.success,
-.table > tbody > tr > th.success,
-.table > tfoot > tr > th.success,
-.table > thead > tr.success > td,
-.table > tbody > tr.success > td,
-.table > tfoot > tr.success > td,
-.table > thead > tr.success > th,
-.table > tbody > tr.success > th,
-.table > tfoot > tr.success > th {
-  background-color: #dff0d8;
-}
-.table-hover > tbody > tr > td.success:hover,
-.table-hover > tbody > tr > th.success:hover,
-.table-hover > tbody > tr.success:hover > td,
-.table-hover > tbody > tr:hover > .success,
-.table-hover > tbody > tr.success:hover > th {
-  background-color: #d0e9c6;
-}
-.table > thead > tr > td.info,
-.table > tbody > tr > td.info,
-.table > tfoot > tr > td.info,
-.table > thead > tr > th.info,
-.table > tbody > tr > th.info,
-.table > tfoot > tr > th.info,
-.table > thead > tr.info > td,
-.table > tbody > tr.info > td,
-.table > tfoot > tr.info > td,
-.table > thead > tr.info > th,
-.table > tbody > tr.info > th,
-.table > tfoot > tr.info > th {
-  background-color: #d9edf7;
-}
-.table-hover > tbody > tr > td.info:hover,
-.table-hover > tbody > tr > th.info:hover,
-.table-hover > tbody > tr.info:hover > td,
-.table-hover > tbody > tr:hover > .info,
-.table-hover > tbody > tr.info:hover > th {
-  background-color: #c4e3f3;
-}
-.table > thead > tr > td.warning,
-.table > tbody > tr > td.warning,
-.table > tfoot > tr > td.warning,
-.table > thead > tr > th.warning,
-.table > tbody > tr > th.warning,
-.table > tfoot > tr > th.warning,
-.table > thead > tr.warning > td,
-.table > tbody > tr.warning > td,
-.table > tfoot > tr.warning > td,
-.table > thead > tr.warning > th,
-.table > tbody > tr.warning > th,
-.table > tfoot > tr.warning > th {
-  background-color: #fcf8e3;
-}
-.table-hover > tbody > tr > td.warning:hover,
-.table-hover > tbody > tr > th.warning:hover,
-.table-hover > tbody > tr.warning:hover > td,
-.table-hover > tbody > tr:hover > .warning,
-.table-hover > tbody > tr.warning:hover > th {
-  background-color: #faf2cc;
-}
-.table > thead > tr > td.danger,
-.table > tbody > tr > td.danger,
-.table > tfoot > tr > td.danger,
-.table > thead > tr > th.danger,
-.table > tbody > tr > th.danger,
-.table > tfoot > tr > th.danger,
-.table > thead > tr.danger > td,
-.table > tbody > tr.danger > td,
-.table > tfoot > tr.danger > td,
-.table > thead > tr.danger > th,
-.table > tbody > tr.danger > th,
-.table > tfoot > tr.danger > th {
-  background-color: #f2dede;
-}
-.table-hover > tbody > tr > td.danger:hover,
-.table-hover > tbody > tr > th.danger:hover,
-.table-hover > tbody > tr.danger:hover > td,
-.table-hover > tbody > tr:hover > .danger,
-.table-hover > tbody > tr.danger:hover > th {
-  background-color: #ebcccc;
-}
-.table-responsive {
-  overflow-x: auto;
-  min-height: 0.01%;
-}
-@media screen and (max-width: 767px) {
-  .table-responsive {
-    width: 100%;
-    margin-bottom: 15px;
-    overflow-y: hidden;
-    -ms-overflow-style: -ms-autohiding-scrollbar;
-    border: 1px solid #ddd;
-  }
-  .table-responsive > .table {
-    margin-bottom: 0;
-  }
-  .table-responsive > .table > thead > tr > th,
-  .table-responsive > .table > tbody > tr > th,
-  .table-responsive > .table > tfoot > tr > th,
-  .table-responsive > .table > thead > tr > td,
-  .table-responsive > .table > tbody > tr > td,
-  .table-responsive > .table > tfoot > tr > td {
-    white-space: nowrap;
-  }
-  .table-responsive > .table-bordered {
-    border: 0;
-  }
-  .table-responsive > .table-bordered > thead > tr > th:first-child,
-  .table-responsive > .table-bordered > tbody > tr > th:first-child,
-  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
-  .table-responsive > .table-bordered > thead > tr > td:first-child,
-  .table-responsive > .table-bordered > tbody > tr > td:first-child,
-  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
-    border-left: 0;
-  }
-  .table-responsive > .table-bordered > thead > tr > th:last-child,
-  .table-responsive > .table-bordered > tbody > tr > th:last-child,
-  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
-  .table-responsive > .table-bordered > thead > tr > td:last-child,
-  .table-responsive > .table-bordered > tbody > tr > td:last-child,
-  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
-    border-right: 0;
-  }
-  .table-responsive > .table-bordered > tbody > tr:last-child > th,
-  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
-  .table-responsive > .table-bordered > tbody > tr:last-child > td,
-  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
-    border-bottom: 0;
-  }
-}
-fieldset {
-  padding: 0;
-  margin: 0;
-  border: 0;
-  min-width: 0;
-}
-legend {
-  display: block;
-  width: 100%;
-  padding: 0;
-  margin-bottom: 20px;
-  font-size: 21px;
-  line-height: inherit;
-  color: #333333;
-  border: 0;
-  border-bottom: 1px solid #e5e5e5;
-}
-label {
-  display: inline-block;
-  max-width: 100%;
-  margin-bottom: 5px;
-  font-weight: bold;
-}
-input[type="search"] {
-  -webkit-box-sizing: border-box;
-  -moz-box-sizing: border-box;
-  box-sizing: border-box;
-}
-input[type="radio"],
-input[type="checkbox"] {
-  margin: 4px 0 0;
-  margin-top: 1px \9;
-  line-height: normal;
-}
-input[type="file"] {
-  display: block;
-}
-input[type="range"] {
-  display: block;
-  width: 100%;
-}
-select[multiple],
-select[size] {
-  height: auto;
-}
-input[type="file"]:focus,
-input[type="radio"]:focus,
-input[type="checkbox"]:focus {
-  outline: thin dotted;
-  outline: 5px auto -webkit-focus-ring-color;
-  outline-offset: -2px;
-}
-output {
-  display: block;
-  padding-top: 7px;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #555555;
-}
-.form-control {
-  display: block;
-  width: 100%;
-  height: 34px;
-  padding: 6px 12px;
-  font-size: 14px;
-  line-height: 1.42857143;
-  color: #555555;
-  background-color: #fff;
-  background-image: none;
-  border: 1px solid #ccc;
-  border-radius: 4px;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-  -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-  transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-}
-.form-control:focus {
-  border-color: #66afe9;
-  outline: 0;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
-  box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
-}
-.form-control::-moz-placeholder {
-  color: #999;
-  opacity: 1;
-}
-.form-control:-ms-input-placeholder {
-  color: #999;
-}
-.form-control::-webkit-input-placeholder {
-  color: #999;
-}
-.form-control[disabled],
-.form-control[readonly],
-fieldset[disabled] .form-control {
-  background-color: #eeeeee;
-  opacity: 1;
-}
-.form-control[disabled],
-fieldset[disabled] .form-control {
-  cursor: not-allowed;
-}
-textarea.form-control {
-  height: auto;
-}
-input[type="search"] {
-  -webkit-appearance: none;
-}
-@media screen and (-webkit-min-device-pixel-ratio: 0) {
-  input[type="date"].form-control,
-  input[type="time"].form-control,
-  input[type="datetime-local"].form-control,
-  input[type="month"].form-control {
-    line-height: 34px;
-  }
-  input[type="date"].input-sm,
-  input[type="time"].input-sm,
-  input[type="datetime-local"].input-sm,
-  input[type="month"].input-sm,
-  .input-group-sm input[type="date"],
-  .input-group-sm input[type="time"],
-  .input-group-sm input[type="datetime-local"],
-  .input-group-sm input[type="month"] {
-    line-height: 30px;
-  }
-  input[type="date"].input-lg,
-  input[type="time"].input-lg,
-  input[type="datetime-local"].input-lg,
-  input[type="month"].input-lg,
-  .input-group-lg input[type="date"],
-  .input-group-lg input[type="time"],
-  .input-group-lg input[type="datetime-local"],
-  .input-group-lg input[type="month"] {
-    line-height: 46px;
-  }
-}
-.form-group {
-  margin-bottom: 15px;
-}
-.radio,
-.checkbox {
-  position: relative;
-  display: block;
-  margin-top: 10px;
-  margin-bottom: 10px;
-}
-.radio label,
-.checkbox label {
-  min-height: 20px;
-  padding-left: 20px;
-  margin-bottom: 0;
-  font-weight: normal;
-  cursor: pointer;
-}
-.radio input[type="radio"],
-.radio-inline input[type="radio"],
-.checkbox input[type="checkbox"],
-.checkbox-inline input[type="checkbox"] {
-  position: absolute;
-  margin-left: -20px;
-  margin-top: 4px \9;
-}
-.radio + .radio,
-.checkbox + .checkbox {
-  margin-top: -5px;
-}
-.radio-inline,
-.checkbox-inline {
-  position: relative;
-  display: inline-block;
-  padding-left: 20px;
-  margin-bottom: 0;
-  vertical-align: middle;
-  font-weight: normal;
-  cursor: pointer;
-}
-.radio-inline + .radio-inline,
-.checkbox-inline + .checkbox-inline {
-  margin-top: 0;
-  margin-left: 10px;
-}
-input[type="radio"][disabled],
-input[type="checkbox"][disabled],
-input[type="radio"].disabled,
-input[type="checkbox"].disabled,
-fieldset[disabled] input[type="radio"],
-fieldset[disabled] input[type="checkbox"] {
-  cursor: not-allowed;
-}
-.radio-inline.disabled,
-.checkbox-inline.disabled,
-fieldset[disabled] .radio-inline,
-fieldset[disabled] .checkbox-inline {
-  cursor: not-allowed;
-}
-.radio.disabled label,
-.checkbox.disabled label,
-fieldset[disabled] .radio label,
-fieldset[disabled] .checkbox label {
-  cursor: not-allowed;
-}
-.form-control-static {
-  padding-top: 7px;
-  padding-bottom: 7px;
-  margin-bottom: 0;
-  min-height: 34px;
-}
-.form-control-static.input-lg,
-.form-control-static.input-sm {
-  padding-left: 0;
-  padding-right: 0;
-}
-.input-sm {
-  height: 30px;
-  padding: 5px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-  border-radius: 3px;
-}
-select.input-sm {
-  height: 30px;
-  line-height: 30px;
-}
-textarea.input-sm,
-select[multiple].input-sm {
-  height: auto;
-}
-.form-group-sm .form-control {
-  height: 30px;
-  padding: 5px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-  border-radius: 3px;
-}
-.form-group-sm select.form-control {
-  height: 30px;
-  line-height: 30px;
-}
-.form-group-sm textarea.form-control,
-.form-group-sm select[multiple].form-control {
-  height: auto;
-}
-.form-group-sm .form-control-static {
-  height: 30px;
-  min-height: 32px;
-  padding: 6px 10px;
-  font-size: 12px;
-  line-height: 1.5;
-}
-.input-lg {
-  height: 46px;
-  padding: 10px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-  border-radius: 6px;
-}
-select.input-lg {
-  height: 46px;
-  line-height: 46px;
-}
-textarea.input-lg,
-select[multiple].input-lg {
-  height: auto;
-}
-.form-group-lg .form-control {
-  height: 46px;
-  padding: 10px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-  border-radius: 6px;
-}
-.form-group-lg select.form-control {
-  height: 46px;
-  line-height: 46px;
-}
-.form-group-lg textarea.form-control,
-.form-group-lg select[multiple].form-control {
-  height: auto;
-}
-.form-group-lg .form-control-static {
-  height: 46px;
-  min-height: 38px;
-  padding: 11px 16px;
-  font-size: 18px;
-  line-height: 1.3333333;
-}
-.has-feedback {
-  position: relative;
-}
-.has-feedback .form-control {
-  padding-right: 42.5px;
-}
-.form-control-feedback {
-  position: absolute;
-  top: 0;
-  right: 0;
-  z-index: 2;
-  display: block;
-  width: 34px;
-  height: 34px;
-  line-height: 34px;
-  text-align: center;
-  pointer-events: none;
-}
-.input-lg + .form-control-feedback,
-.input-group-lg + .form-control-feedback,
-.form-group-lg .form-control + .form-control-feedback {
-  width: 46px;
-  height: 46px;
-  line-height: 46px;
-}
-.input-sm + .form-control-feedback,
-.input-group-sm + .form-control-feedback,
-.form-group-sm .form-control + .form-control-feedback {
-  width: 30px;
-  height: 30px;
-  line-height: 30px;
-}
-.has-success .help-block,
-.has-success .control-label,
-.has-success .radio,
-.has-success .checkbox,
-.has-success .radio-inline,
-.has-success .checkbox-inline,
-.has-success.radio label,
-.has-success.checkbox label,
-.has-success.radio-inline label,
-.has-success.checkbox-inline label {
-  color: #3c763d;
-}
-.has-success .form-control {
-  border-color: #3c763d;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-success .form-control:focus {
-  border-color: #2b542c;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
-}
-.has-success .input-group-addon {
-  color: #3c763d;
-  border-color: #3c763d;
-  background-color: #dff0d8;
-}
-.has-success .form-control-feedback {
-  color: #3c763d;
-}
-.has-warning .help-block,
-.has-warning .control-label,
-.has-warning .radio,
-.has-warning .checkbox,
-.has-warning .radio-inline,
-.has-warning .checkbox-inline,
-.has-warning.radio label,
-.has-warning.checkbox label,
-.has-warning.radio-inline label,
-.has-warning.checkbox-inline label {
-  color: #8a6d3b;
-}
-.has-warning .form-control {
-  border-color: #8a6d3b;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-warning .form-control:focus {
-  border-color: #66512c;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
-}
-.has-warning .input-group-addon {
-  color: #8a6d3b;
-  border-color: #8a6d3b;
-  background-color: #fcf8e3;
-}
-.has-warning .form-control-feedback {
-  color: #8a6d3b;
-}
-.has-error .help-block,
-.has-error .control-label,
-.has-error .radio,
-.has-error .checkbox,
-.has-error .radio-inline,
-.has-error .checkbox-inline,
-.has-error.radio label,
-.has-error.checkbox label,
-.has-error.radio-inline label,
-.has-error.checkbox-inline label {
-  color: #a94442;
-}
-.has-error .form-control {
-  border-color: #a94442;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
-}
-.has-error .form-control:focus {
-  border-color: #843534;
-  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
-  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
-}
-.has-error .input-group-addon {
-  color: #a94442;
-  border-color: #a94442;
-  background-color: #f2dede;
-}
-.has-error .form-control-feedback {
-  color: #a94442;
-}
-.has-feedback label ~ .form-control-feedback {
-  top: 25px;
-}
-.has-feedback label.sr-only ~ .form-control-feedback {
-  top: 0;
-}
-.help-block {
-  display: block;
-  margin-top: 5px;
-  margin-bottom: 10px;
-  color: #737373;
-}
-@media (min-width: 768px) {
-  .form-inline .form-group {
-    display: inline-block;
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .form-control {
-    display: inline-block;
-    width: auto;
-    vertical-align: middle;
-  }
-  .form-inline .form-control-static {
-    display: inline-block;
-  }
-  .form-inline .input-group {
-    display: inline-table;
-    vertical-align: middle;
-  }
-  .form-inline .input-group .input-group-addon,
-  .form-inline .input-group .input-group-btn,
-  .form-inline .input-group .form-control {
-    width: auto;
-  }
-  .form-inline .input-group > .form-control {
-    width: 100%;
-  }
-  .form-inline .control-label {
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .radio,
-  .form-inline .checkbox {
-    display: inline-block;
-    margin-top: 0;
-    margin-bottom: 0;
-    vertical-align: middle;
-  }
-  .form-inline .radio label,
-  .form-inline .checkbox label {
-    padding-left: 0;
-  }
-  .form-inline .radio input[type="radio"],
-  .form-inline .checkbox input[type="checkbox"] {
-    position: relative;
-    margin-left: 0;
-  }
-  .form-inline .has-feedback .form-control-feedback {
-    top: 0;
-  }
-}
-.form-horizontal .radio,
-.form-horizontal .checkbox,
-.form-horizontal .radio-inline,
-.form-horizontal .checkbox-inline {
-  margin-top: 0;
-  margin-bottom: 0;
-  padding-top: 7px;
-}
-.form-horizontal .radio,
-.form-horizontal .checkbox {
-  min-height: 27px;
-}
-.form-horizontal .form-group {
-  margin-left: -15px;
-  margin-right: -15px;
-}
-@media (min-width: 768px) {
-  .form-horizontal .control-label {
-    text-align: right;
-    margin-bottom: 0;
-    padding-top: 7px;
-  }
-}
-.form-horizontal .has-feedback .form-control-feedback {
-  right: 15px;
-}
-@media (min-width: 768px) {
-  .form-horizontal .form-group-lg .control-label {
-    padding-top: 14.333333px;
-    font-size: 18px;
-  }
-}
-@media (min-width: 768px) {
-  .form-horizontal .form-group-sm .control-label {
-    padding-top: 6px;
-    font-size: 12px;
-  }
-}
-.btn {
-  display: inline-block;
-  margin-bottom: 0;
-  font-weight: normal;
-  text-align: center;
-  vertical-align: middle;
-  touch-action: manipulation;
-  cursor: pointer;
-  background-image: none;
-  border: 1px solid transparent;
- 

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf b/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf
deleted file mode 100644
index f7936cc..0000000
Binary files a/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf and /dev/null differ


[09/47] flink git commit: [FLINK-2805] [blobmanager] Write JARs to file state backend for recovery

Posted by tr...@apache.org.
[FLINK-2805] [blobmanager] Write JARs to file state backend for recovery

Move StateBackend enum to top level and org.apache.flink.runtime.state

Abstract blob store in blob server for recovery

This closes #1227.


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

Branch: refs/heads/master
Commit: c3a4d1d9f720a1da9697d0bbf48f7a3b1f5851b8
Parents: c2989f2
Author: Ufuk Celebi <uc...@apache.org>
Authored: Mon Oct 5 14:30:46 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:52 2015 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/blob/BlobServer.java   | 105 +++++++++--
 .../runtime/blob/BlobServerConnection.java      |  52 ++++--
 .../apache/flink/runtime/blob/BlobStore.java    |  97 ++++++++++
 .../apache/flink/runtime/blob/BlobUtils.java    |  75 +++++++-
 .../flink/runtime/blob/FileSystemBlobStore.java | 186 +++++++++++++++++++
 .../flink/runtime/blob/VoidBlobStore.java       |  61 ++++++
 .../flink/runtime/jobmanager/RecoveryMode.java  |  12 +-
 .../flink/runtime/blob/BlobRecoveryITCase.java  | 159 ++++++++++++++++
 .../BlobLibraryCacheRecoveryITCase.java         | 176 ++++++++++++++++++
 9 files changed, 874 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/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 ef2ef61..d0bed8c 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
@@ -18,6 +18,14 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -30,13 +38,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.api.common.JobID;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
  * This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and
@@ -57,12 +59,12 @@ public class BlobServer extends Thread implements BlobService {
 	/** Indicates whether a shutdown of server component has been requested. */
 	private final AtomicBoolean shutdownRequested = new AtomicBoolean();
 
-	/** Shutdown hook thread to ensure deletion of the storage directory. */
-	private final Thread shutdownHook;
-
 	/** Is the root directory for file storage */
 	private final File storageDir;
 
+	/** Blob store for recovery */
+	private final BlobStore blobStore;
+
 	/** Set of currently running threads */
 	private final Set<BlobServerConnection> activeConnections = new HashSet<BlobServerConnection>();
 
@@ -70,18 +72,43 @@ public class BlobServer extends Thread implements BlobService {
 	private final int maxConnections;
 
 	/**
+	 * Shutdown hook thread to ensure deletion of the storage directory (or <code>null</code> if
+	 * the configured recovery mode does not equal{@link RecoveryMode#STANDALONE})
+	 */
+	private final Thread shutdownHook;
+
+	/**
 	 * Instantiates a new BLOB server and binds it to a free network port.
-	 * 
+	 *
 	 * @throws IOException
 	 *         thrown if the BLOB server cannot bind to a free network port
 	 */
 	public BlobServer(Configuration config) throws IOException {
+		checkNotNull(config, "Configuration");
+
+		RecoveryMode recoveryMode = RecoveryMode.fromConfig(config);
 
 		// configure and create the storage directory
 		String storageDirectory = config.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
 		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB server storage directory {}", storageDir);
 
+		// No recovery.
+		if (recoveryMode == RecoveryMode.STANDALONE) {
+			this.blobStore = new VoidBlobStore();
+		}
+		// Recovery. Check that everything has been setup correctly. This is not clean, but it's
+		// better to resolve this with some upcoming changes to the state backend setup.
+		else if (config.containsKey(ConfigConstants.STATE_BACKEND) &&
+				config.containsKey(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH)) {
+
+			this.blobStore = new FileSystemBlobStore(config);
+		}
+		// Fallback.
+		else {
+			this.blobStore = new VoidBlobStore();
+		}
+
 		// configure the maximum number of concurrent connections
 		final int maxConnections = config.getInteger(
 				ConfigConstants.BLOB_FETCH_CONCURRENT_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT);
@@ -102,8 +129,13 @@ public class BlobServer extends Thread implements BlobService {
 			backlog = ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG;
 		}
 
-		// Add shutdown hook to delete storage directory
-		this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
+		if (recoveryMode == RecoveryMode.STANDALONE) {
+			// Add shutdown hook to delete storage directory
+			this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
+		}
+		else {
+			this.shutdownHook = null;
+		}
 
 		// start the server
 		try {
@@ -132,37 +164,43 @@ public class BlobServer extends Thread implements BlobService {
 	 * Returns a file handle to the file associated with the given blob key on the blob
 	 * server.
 	 *
+	 * <p><strong>This is only called from the {@link BlobServerConnection}</strong>
+	 *
 	 * @param key identifying the file
 	 * @return file handle to the file
 	 */
-	public File getStorageLocation(BlobKey key) {
+	File getStorageLocation(BlobKey key) {
 		return BlobUtils.getStorageLocation(storageDir, key);
 	}
 
 	/**
 	 * Returns a file handle to the file identified by the given jobID and key.
 	 *
+	 * <p><strong>This is only called from the {@link BlobServerConnection}</strong>
+	 *
 	 * @param jobID to which the file is associated
 	 * @param key to identify the file within the job context
 	 * @return file handle to the file
 	 */
-	public File getStorageLocation(JobID jobID, String key) {
+	File getStorageLocation(JobID jobID, String key) {
 		return BlobUtils.getStorageLocation(storageDir, jobID, key);
 	}
 
 	/**
 	 * Method which deletes all files associated with the given jobID.
 	 *
+	 * <p><strong>This is only called from the {@link BlobServerConnection}</strong>
+	 *
 	 * @param jobID all files associated to this jobID will be deleted
 	 * @throws IOException
 	 */
-	public void deleteJobDirectory(JobID jobID) throws IOException {
+	void deleteJobDirectory(JobID jobID) throws IOException {
 		BlobUtils.deleteJobDirectory(storageDir, jobID);
 	}
 
 	/**
 	 * Returns a temporary file inside the BLOB server's incoming directory.
-	 * 
+	 *
 	 * @return a temporary file inside the BLOB server's incoming directory
 	 */
 	File createTemporaryFilename() {
@@ -170,6 +208,13 @@ public class BlobServer extends Thread implements BlobService {
 				String.format("temp-%08d", tempFileCounter.getAndIncrement()));
 	}
 
+	/**
+	 * Returns the blob store.
+	 */
+	BlobStore getBlobStore() {
+		return blobStore;
+	}
+
 	@Override
 	public void run() {
 		try {
@@ -245,6 +290,9 @@ public class BlobServer extends Thread implements BlobService {
 				LOG.error("BLOB server failed to properly clean up its storage directory.");
 			}
 
+			// Clean up the recovery directory
+			blobStore.cleanUp();
+
 			// Remove shutdown hook to prevent resource leaks, unless this is invoked by the
 			// shutdown hook itself
 			if (shutdownHook != null && shutdownHook != Thread.currentThread()) {
@@ -282,11 +330,26 @@ public class BlobServer extends Thread implements BlobService {
 
 		final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob);
 
-		if (!localFile.exists()) {
-			throw new FileNotFoundException("File " + localFile.getCanonicalPath() + " does not exist.");
-		} else {
+		if (localFile.exists()) {
 			return localFile.toURI().toURL();
 		}
+		else {
+			try {
+				// Try the blob store
+				blobStore.get(requiredBlob, localFile);
+			}
+			catch (Exception e) {
+				throw new IOException("Failed to copy from blob store.", e);
+			}
+
+			if (localFile.exists()) {
+				return localFile.toURI().toURL();
+			}
+			else {
+				throw new FileNotFoundException("Local file " + localFile + " does not exist " +
+						"and failed to copy from blob store.");
+			}
+		}
 	}
 
 	/**
@@ -305,6 +368,8 @@ public class BlobServer extends Thread implements BlobService {
 				LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath());
 			}
 		}
+
+		blobStore.delete(key);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/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 793a9d6..d7bba8f 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
@@ -18,6 +18,12 @@
 
 package org.apache.flink.runtime.blob;
 
+import com.google.common.io.Files;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.util.InstantiationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
@@ -29,28 +35,21 @@ import java.net.Socket;
 import java.net.SocketException;
 import java.security.MessageDigest;
 
-import com.google.common.io.Files;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.util.InstantiationUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_ID_SCOPE;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.MAX_KEY_LENGTH;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.NAME_ADDRESSABLE;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY;
 import static org.apache.flink.runtime.blob.BlobUtils.closeSilently;
 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.runtime.blob.BlobServerProtocol.BUFFER_SIZE;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.MAX_KEY_LENGTH;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR;
-
 /**
  * A BLOB connection handles a series of requests from a particular BLOB client.
  */
@@ -181,10 +180,18 @@ class BlobServerConnection extends Thread {
 				JobID jobID = JobID.fromByteArray(jidBytes);
 				String key = readKey(buf, inputStream);
 				blobFile = this.blobServer.getStorageLocation(jobID, key);
+
+				if (!blobFile.exists()) {
+					blobServer.getBlobStore().get(jobID, key, blobFile);
+				}
 			}
 			else if (contentAddressable == CONTENT_ADDRESSABLE) {
 				final BlobKey key = BlobKey.readFromInputStream(inputStream);
 				blobFile = blobServer.getStorageLocation(key);
+
+				if (!blobFile.exists()) {
+					blobServer.getBlobStore().get(key, blobFile);
+				}
 			}
 			else {
 				throw new IOException("Unknown type of BLOB addressing.");
@@ -194,6 +201,7 @@ class BlobServerConnection extends Thread {
 			if (!blobFile.exists()) {
 				throw new IOException("Cannot find required BLOB at " + blobFile.getAbsolutePath());
 			}
+
 			if (blobFile.length() > Integer.MAX_VALUE) {
 				throw new IOException("BLOB size exceeds the maximum size (2 GB).");
 			}
@@ -220,8 +228,7 @@ class BlobServerConnection extends Thread {
 			int blobLen = (int) blobFile.length();
 			writeLength(blobLen, outputStream);
 
-			FileInputStream fis = new FileInputStream(blobFile);
-			try {
+			try (FileInputStream fis = new FileInputStream(blobFile)) {
 				int bytesRemaining = blobLen;
 				while (bytesRemaining > 0) {
 					int read = fis.read(buf);
@@ -231,8 +238,6 @@ class BlobServerConnection extends Thread {
 					outputStream.write(buf, 0, read);
 					bytesRemaining -= read;
 				}
-			} finally {
-				fis.close();
 			}
 		}
 		catch (SocketException e) {
@@ -314,6 +319,9 @@ class BlobServerConnection extends Thread {
 				File storageFile = this.blobServer.getStorageLocation(jobID, key);
 				Files.move(incomingFile, storageFile);
 				incomingFile = null;
+
+				blobServer.getBlobStore().put(storageFile, jobID, key);
+
 				outputStream.write(RETURN_OKAY);
 			}
 			else {
@@ -322,6 +330,8 @@ class BlobServerConnection extends Thread {
 				Files.move(incomingFile, storageFile);
 				incomingFile = null;
 
+				blobServer.getBlobStore().put(storageFile, blobKey);
+
 				// Return computed key to client for validation
 				outputStream.write(RETURN_OKAY);
 				blobKey.writeToOutputStream(outputStream);
@@ -379,6 +389,8 @@ class BlobServerConnection extends Thread {
 				if (blobFile.exists() && !blobFile.delete()) {
 					throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath());
 				}
+
+				blobServer.getBlobStore().delete(key);
 			}
 			else if (type == NAME_ADDRESSABLE) {
 				byte[] jidBytes = new byte[JobID.SIZE];
@@ -391,6 +403,8 @@ class BlobServerConnection extends Thread {
 				if (blobFile.exists() && !blobFile.delete()) {
 					throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath());
 				}
+
+				blobServer.getBlobStore().delete(jobID, key);
 			}
 			else if (type == JOB_ID_SCOPE) {
 				byte[] jidBytes = new byte[JobID.SIZE];
@@ -398,6 +412,8 @@ class BlobServerConnection extends Thread {
 				JobID jobID = JobID.fromByteArray(jidBytes);
 
 				blobServer.deleteJobDirectory(jobID);
+
+				blobServer.getBlobStore().deleteAll(jobID);
 			}
 			else {
 				throw new IOException("Unrecognized addressing type: " + type);

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
new file mode 100644
index 0000000..1e72d91
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.blob;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.File;
+
+/**
+ * A blob store.
+ */
+interface BlobStore {
+
+	/**
+	 * Copies the local file to the blob store.
+	 *
+	 * @param localFile The file to copy
+	 * @param blobKey   The ID for the file in the blob store
+	 * @throws Exception If the copy fails
+	 */
+	void put(File localFile, BlobKey blobKey) throws Exception;
+
+	/**
+	 * Copies a local file to the blob store.
+	 *
+	 * <p>The job ID and key make up a composite key for the file.
+	 *
+	 * @param localFile The file to copy
+	 * @param jobId     The JobID part of ID for the file in the blob store
+	 * @param key       The String part of ID for the file in the blob store
+	 * @throws Exception If the copy fails
+	 */
+	void put(File localFile, JobID jobId, String key) throws Exception;
+
+	/**
+	 * Copies a blob to a local file.
+	 *
+	 * @param blobKey   The blob ID
+	 * @param localFile The local file to copy to
+	 * @throws Exception If the copy fails
+	 */
+	void get(BlobKey blobKey, File localFile) throws Exception;
+
+	/**
+	 * Copies a blob to a local file.
+	 *
+	 * @param jobId     The JobID part of ID for the blob
+	 * @param key       The String part of ID for the blob
+	 * @param localFile The local file to copy to
+	 * @throws Exception If the copy fails
+	 */
+	void get(JobID jobId, String key, File localFile) throws Exception;
+
+	/**
+	 * Deletes a blob.
+	 *
+	 * @param blobKey The blob ID
+	 */
+	void delete(BlobKey blobKey);
+
+	/**
+	 * Deletes a blob.
+	 *
+	 * @param jobId The JobID part of ID for the blob
+	 * @param key   The String part of ID for the blob
+	 */
+	void delete(JobID jobId, String key);
+
+	/**
+	 * Deletes blobs.
+	 *
+	 * @param jobId The JobID part of all blobs to delete
+	 */
+	void deleteAll(JobID jobId);
+
+	/**
+	 * Cleans up the store and deletes all blobs.
+	 */
+	void cleanUp();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/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 c47ecf2..d8f744b 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
@@ -21,14 +21,19 @@ package org.apache.flink.runtime.blob;
 import com.google.common.io.BaseEncoding;
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.util.IOUtils;
 import org.slf4j.Logger;
 
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.Socket;
+import java.net.URI;
 import java.nio.charset.Charset;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -49,12 +54,12 @@ public class BlobUtils {
 	/**
 	 * The prefix of all BLOB files stored by the BLOB server.
 	 */
-	private static final String BLOB_FILE_PREFIX = "blob_";
+	static final String BLOB_FILE_PREFIX = "blob_";
 
 	/**
 	 * The prefix of all job-specific directories created by the BLOB server.
 	 */
-	private static final String JOB_DIR_PREFIX = "job_";
+	static final String JOB_DIR_PREFIX = "job_";
 
 	/**
 	 * The default character set to translate between characters and bytes.
@@ -103,7 +108,7 @@ public class BlobUtils {
 	static File getIncomingDirectory(File storageDir) {
 		final File incomingDir = new File(storageDir, "incoming");
 
-		if (!incomingDir.exists() && !incomingDir.mkdir()) {
+		if (!incomingDir.exists() && !incomingDir.mkdirs()) {
 			throw new RuntimeException("Cannot create directory for incoming files " + incomingDir.getAbsolutePath());
 		}
 
@@ -119,7 +124,7 @@ public class BlobUtils {
 	private static File getCacheDirectory(File storageDir) {
 		final File cacheDirectory = new File(storageDir, "cache");
 
-		if (!cacheDirectory.exists() && !cacheDirectory.mkdir()) {
+		if (!cacheDirectory.exists() && !cacheDirectory.mkdirs()) {
 			throw new RuntimeException("Could not create cache directory '" + cacheDirectory.getAbsolutePath() + "'.");
 		}
 
@@ -174,7 +179,7 @@ public class BlobUtils {
 	 *        the user's key for a BLOB
 	 * @return the internal name for the BLOB as used by the BLOB server
 	 */
-	private static String encodeKey(String key) {
+	static String encodeKey(String key) {
 		return BaseEncoding.base64().encode(key.getBytes(DEFAULT_CHARSET));
 	}
 
@@ -327,6 +332,66 @@ public class BlobUtils {
 	}
 
 	/**
+	 * Returns the path for the given blob key.
+	 *
+	 * <p>The returned path can be used with the state backend for recovery purposes.
+	 *
+	 * <p>This follows the same scheme as {@link #getStorageLocation(File, BlobKey)}.
+	 */
+	static String getRecoveryPath(String basePath, BlobKey blobKey) {
+		// format: $base/cache/blob_$key
+		return String.format("%s/cache/%s", basePath, BLOB_FILE_PREFIX + blobKey.toString());
+	}
+
+	/**
+	 * Returns the path for the given job ID and key.
+	 *
+	 * <p>The returned path can be used with the state backend for recovery purposes.
+	 *
+	 * <p>This follows the same scheme as {@link #getStorageLocation(File, JobID, String)}.
+	 */
+	static String getRecoveryPath(String basePath, JobID jobId, String key) {
+		// format: $base/job_$id/blob_$key
+		return String.format("%s/%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString(),
+				BLOB_FILE_PREFIX + encodeKey(key));
+	}
+
+	/**
+	 * Returns the path for the given job ID.
+	 *
+	 * <p>The returned path can be used with the state backend for recovery purposes.
+	 */
+	static String getRecoveryPath(String basePath, JobID jobId) {
+		return String.format("%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString());
+	}
+
+	/**
+	 * Copies the file from the recovery path to the local file.
+	 */
+	static void copyFromRecoveryPath(String recoveryPath, File localBlobFile) throws Exception {
+		if (recoveryPath == null) {
+			throw new IllegalStateException("Failed to determine recovery path.");
+		}
+
+		if (!localBlobFile.createNewFile()) {
+			throw new IllegalStateException("Failed to create new local file to copy to");
+		}
+
+		URI uri = new URI(recoveryPath);
+		Path path = new Path(recoveryPath);
+
+		if (FileSystem.get(uri).exists(path)) {
+			try (InputStream is = FileSystem.get(uri).open(path)) {
+				FileOutputStream fos = new FileOutputStream(localBlobFile);
+				IOUtils.copyBytes(is, fos); // closes the streams
+			}
+		}
+		else {
+			throw new IOException("Cannot find required BLOB at '" + recoveryPath + "' for recovery.");
+		}
+	}
+
+	/**
 	 * Private constructor to prevent instantiation.
 	 */
 	private BlobUtils() {

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
new file mode 100644
index 0000000..8a037ad
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -0,0 +1,186 @@
+/*
+ * 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.blob;
+
+import com.google.common.io.Files;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Blob store backed by {@link FileSystem}.
+ */
+class FileSystemBlobStore implements BlobStore {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FileSystemBlobStore.class);
+
+	/** The base path of the blob store */
+	private final String basePath;
+
+	FileSystemBlobStore(Configuration config) throws IOException {
+		StateBackend stateBackend = StateBackend.fromConfig(config);
+
+		if (stateBackend == StateBackend.FILESYSTEM) {
+			String stateBackendBasePath = config.getString(
+					ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "");
+
+			if (stateBackendBasePath.equals("")) {
+				throw new IllegalConfigurationException(String.format("Missing configuration for " +
+						"file system state backend recovery path. Please specify via " +
+						"'%s' key.", ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH));
+			}
+
+			stateBackendBasePath += "/blob";
+
+			this.basePath = stateBackendBasePath;
+
+			try {
+				FileSystem.get(new URI(basePath)).mkdirs(new Path(basePath));
+			}
+			catch (URISyntaxException e) {
+				throw new IOException(e);
+			}
+
+			LOG.info("Created blob directory {}.", basePath);
+		}
+		else {
+			// Nothing else support at the moment
+			throw new IllegalConfigurationException(
+					String.format("Illegal state backend " +
+									"configuration '%s'. Please configure '%s' as state " +
+									"backend and specify the recovery path via '%s' key.",
+							stateBackend, StateBackend.FILESYSTEM,
+							ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH));
+		}
+	}
+
+	// - Put ------------------------------------------------------------------
+
+	@Override
+	public void put(File localFile, BlobKey blobKey) throws Exception {
+		put(localFile, BlobUtils.getRecoveryPath(basePath, blobKey));
+	}
+
+	@Override
+	public void put(File localFile, JobID jobId, String key) throws Exception {
+		put(localFile, BlobUtils.getRecoveryPath(basePath, jobId, key));
+	}
+
+	private void put(File fromFile, String toBlobPath) throws Exception {
+		try (OutputStream os = FileSystem.get(new URI(toBlobPath))
+				.create(new Path(toBlobPath), true)) {
+
+			LOG.debug("Copying from {} to {}.", fromFile, toBlobPath);
+			Files.copy(fromFile, os);
+		}
+	}
+
+	// - Get ------------------------------------------------------------------
+
+	@Override
+	public void get(BlobKey blobKey, File localFile) throws Exception {
+		get(BlobUtils.getRecoveryPath(basePath, blobKey), localFile);
+	}
+
+	@Override
+	public void get(JobID jobId, String key, File localFile) throws Exception {
+		get(BlobUtils.getRecoveryPath(basePath, jobId, key), localFile);
+	}
+
+	private void get(String fromBlobPath, File toFile) throws Exception {
+		checkNotNull(fromBlobPath, "Blob path");
+		checkNotNull(toFile, "File");
+
+		if (!toFile.exists() && !toFile.createNewFile()) {
+			throw new IllegalStateException("Failed to create target file to copy to");
+		}
+
+		final URI fromUri = new URI(fromBlobPath);
+		final Path fromPath = new Path(fromBlobPath);
+
+		if (FileSystem.get(fromUri).exists(fromPath)) {
+			try (InputStream is = FileSystem.get(fromUri).open(fromPath)) {
+				FileOutputStream fos = new FileOutputStream(toFile);
+
+				LOG.debug("Copying from {} to {}.", fromBlobPath, toFile);
+				IOUtils.copyBytes(is, fos); // closes the streams
+			}
+		}
+		else {
+			throw new IOException(fromBlobPath + " does not exist.");
+		}
+	}
+
+	// - Delete ---------------------------------------------------------------
+
+	@Override
+	public void delete(BlobKey blobKey) {
+		delete(BlobUtils.getRecoveryPath(basePath, blobKey));
+	}
+
+	@Override
+	public void delete(JobID jobId, String key) {
+		delete(BlobUtils.getRecoveryPath(basePath, jobId, key));
+	}
+
+	@Override
+	public void deleteAll(JobID jobId) {
+		delete(BlobUtils.getRecoveryPath(basePath, jobId));
+	}
+
+	private void delete(String blobPath) {
+		try {
+			LOG.debug("Deleting {}.", blobPath);
+
+			FileSystem.get(new URI(blobPath)).delete(new Path(blobPath), true);
+		}
+		catch (Exception e) {
+			LOG.warn("Failed to delete blob at " + blobPath);
+		}
+	}
+
+	@Override
+	public void cleanUp() {
+		try {
+			LOG.debug("Cleaning up {}.", basePath);
+
+			FileSystem.get(new URI(basePath)).delete(new Path(basePath), true);
+		}
+		catch (Exception e) {
+			LOG.error("Failed to clean up recovery directory.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
new file mode 100644
index 0000000..1b71add
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
@@ -0,0 +1,61 @@
+/*
+ * 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.blob;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.File;
+
+/**
+ * A blob store doing nothing.
+ */
+class VoidBlobStore implements BlobStore {
+
+	@Override
+	public void put(File localFile, BlobKey blobKey) throws Exception {
+	}
+
+	@Override
+	public void put(File localFile, JobID jobId, String key) throws Exception {
+	}
+
+	@Override
+	public void get(BlobKey blobKey, File localFile) throws Exception {
+	}
+
+	@Override
+	public void get(JobID jobId, String key, File localFile) throws Exception {
+	}
+
+	@Override
+	public void delete(BlobKey blobKey) {
+	}
+
+	@Override
+	public void delete(JobID jobId, String key) {
+	}
+
+	@Override
+	public void deleteAll(JobID jobId) {
+	}
+
+	@Override
+	public void cleanUp() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
index 17322d8..077e34d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
@@ -24,11 +24,11 @@ import org.apache.flink.configuration.Configuration;
 /**
  * Recovery mode for Flink's cluster execution. Currently supported modes are:
  *
- *   - Standalone: No recovery from JobManager failures
- *   - ZooKeeper: JobManager high availability via ZooKeeper
- *     ZooKeeper is used to select a leader among a group of JobManager. This JobManager
- *     is responsible for the job execution. Upon failure of the leader a new leader is elected
- *     which will take over the responsibilities of the old leader
+ * - Standalone: No recovery from JobManager failures
+ * - ZooKeeper: JobManager high availability via ZooKeeper
+ * ZooKeeper is used to select a leader among a group of JobManager. This JobManager
+ * is responsible for the job execution. Upon failure of the leader a new leader is elected
+ * which will take over the responsibilities of the old leader
  */
 public enum RecoveryMode {
 	STANDALONE,
@@ -69,4 +69,4 @@ public enum RecoveryMode {
 				return false;
 		}
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
new file mode 100644
index 0000000..0e324a8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.blob;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class BlobRecoveryITCase {
+
+	private File recoveryDir;
+
+	@Before
+	public void setUp() throws Exception {
+		recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir");
+		if (!recoveryDir.exists() && !recoveryDir.mkdirs()) {
+			throw new IllegalStateException("Failed to create temp directory for test");
+		}
+	}
+
+	@After
+	public void cleanUp() throws Exception {
+		if (recoveryDir != null) {
+			FileUtils.deleteDirectory(recoveryDir);
+		}
+	}
+
+	/**
+	 * Tests that with {@link RecoveryMode#ZOOKEEPER} distributed JARs are recoverable from any
+	 * participating BlobServer.
+	 */
+	@Test
+	public void testBlobServerRecovery() throws Exception {
+		Random rand = new Random();
+
+		BlobServer[] server = new BlobServer[2];
+		InetSocketAddress[] serverAddress = new InetSocketAddress[2];
+		BlobClient client = null;
+
+		try {
+			Configuration config = new Configuration();
+			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
+			config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, recoveryDir.getPath());
+
+			for (int i = 0; i < server.length; i++) {
+				server[i] = new BlobServer(config);
+				serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort());
+			}
+
+			client = new BlobClient(serverAddress[0]);
+
+			// Random data
+			byte[] expected = new byte[1024];
+			rand.nextBytes(expected);
+
+			BlobKey[] keys = new BlobKey[2];
+
+			// Put data
+			keys[0] = client.put(expected); // Request 1
+			keys[1] = client.put(expected, 32, 256); // Request 2
+
+			JobID[] jobId = new JobID[] { new JobID(), new JobID() };
+			String[] testKey = new String[] { "test-key-1", "test-key-2" };
+
+			client.put(jobId[0], testKey[0], expected); // Request 3
+			client.put(jobId[1], testKey[1], expected, 32, 256); // Request 4
+
+			// Close the client and connect to the other server
+			client.close();
+			client = new BlobClient(serverAddress[1]);
+
+			// Verify request 1
+			try (InputStream is = client.get(keys[0])) {
+				byte[] actual = new byte[expected.length];
+
+				BlobUtils.readFully(is, actual, 0, expected.length, null);
+
+				for (int i = 0; i < expected.length; i++) {
+					assertEquals(expected[i], actual[i]);
+				}
+			}
+
+			// Verify request 2
+			try (InputStream is = client.get(keys[1])) {
+				byte[] actual = new byte[256];
+				BlobUtils.readFully(is, actual, 0, 256, null);
+
+				for (int i = 32, j = 0; i < 256; i++, j++) {
+					assertEquals(expected[i], actual[j]);
+				}
+			}
+
+			// Verify request 3
+			try (InputStream is = client.get(jobId[0], testKey[0])) {
+				byte[] actual = new byte[expected.length];
+				BlobUtils.readFully(is, actual, 0, expected.length, null);
+
+				for (int i = 0; i < expected.length; i++) {
+					assertEquals(expected[i], actual[i]);
+				}
+			}
+
+			// Verify request 4
+			try (InputStream is = client.get(jobId[1], testKey[1])) {
+				byte[] actual = new byte[256];
+				BlobUtils.readFully(is, actual, 0, 256, null);
+
+				for (int i = 32, j = 0; i < 256; i++, j++) {
+					assertEquals(expected[i], actual[j]);
+				}
+			}
+		}
+		finally {
+			for (BlobServer s : server) {
+				if (s != null) {
+					s.shutdown();
+				}
+			}
+
+			if (client != null) {
+				client.close();
+			}
+		}
+
+		// Verify everything is clean
+		File[] recoveryFiles = recoveryDir.listFiles();
+		assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c3a4d1d9/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
new file mode 100644
index 0000000..4df8afb
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
@@ -0,0 +1,176 @@
+/*
+ * 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.execution.librarycache;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobCache;
+import org.apache.flink.runtime.blob.BlobClient;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class BlobLibraryCacheRecoveryITCase {
+
+	private File recoveryDir;
+
+	@Before
+	public void setUp() throws Exception {
+		recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir");
+		if (!recoveryDir.exists() && !recoveryDir.mkdirs()) {
+			throw new IllegalStateException("Failed to create temp directory for test");
+		}
+	}
+
+	@After
+	public void cleanUp() throws Exception {
+		if (recoveryDir != null) {
+			FileUtils.deleteDirectory(recoveryDir);
+		}
+	}
+
+	/**
+	 * Tests that with {@link RecoveryMode#ZOOKEEPER} distributed JARs are recoverable from any
+	 * participating BlobLibraryCacheManager.
+	 */
+	@Test
+	public void testRecoveryRegisterAndDownload() throws Exception {
+		Random rand = new Random();
+
+		BlobServer[] server = new BlobServer[2];
+		InetSocketAddress[] serverAddress = new InetSocketAddress[2];
+		BlobLibraryCacheManager[] libServer = new BlobLibraryCacheManager[2];
+		BlobCache cache = null;
+		BlobLibraryCacheManager libCache = null;
+
+		try {
+			Configuration config = new Configuration();
+			config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER");
+			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
+			config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, recoveryDir.getPath());
+
+			for (int i = 0; i < server.length; i++) {
+				server[i] = new BlobServer(config);
+				serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort());
+				libServer[i] = new BlobLibraryCacheManager(server[i], 3600 * 1000);
+			}
+
+			// Random data
+			byte[] expected = new byte[1024];
+			rand.nextBytes(expected);
+
+			List<BlobKey> keys = new ArrayList<>(2);
+
+			// Upload some data (libraries)
+			try (BlobClient client = new BlobClient(serverAddress[0])) {
+				keys.add(client.put(expected)); // Request 1
+				keys.add(client.put(expected, 32, 256)); // Request 2
+			}
+
+			// The cache
+			cache = new BlobCache(serverAddress[0], config);
+			libCache = new BlobLibraryCacheManager(cache, 3600 * 1000);
+
+			// Register uploaded libraries
+			JobID jobId = new JobID();
+			ExecutionAttemptID executionId = new ExecutionAttemptID();
+			libServer[0].registerTask(jobId, executionId, keys, Collections.<URL>emptyList());
+
+			// Verify key 1
+			File f = libCache.getFile(keys.get(0));
+			assertEquals(expected.length, f.length());
+
+			try (FileInputStream fis = new FileInputStream(f)) {
+				for (int i = 0; i < expected.length && fis.available() > 0; i++) {
+					assertEquals(expected[i], (byte) fis.read());
+				}
+
+				assertEquals(0, fis.available());
+			}
+
+			// Shutdown cache and start with other server
+			cache.shutdown();
+			libCache.shutdown();
+
+			cache = new BlobCache(serverAddress[1], config);
+			libCache = new BlobLibraryCacheManager(cache, 3600 * 1000);
+
+			// Verify key 1
+			f = libCache.getFile(keys.get(0));
+			assertEquals(expected.length, f.length());
+
+			try (FileInputStream fis = new FileInputStream(f)) {
+				for (int i = 0; i < expected.length && fis.available() > 0; i++) {
+					assertEquals(expected[i], (byte) fis.read());
+				}
+
+				assertEquals(0, fis.available());
+			}
+
+			// Verify key 2
+			f = libCache.getFile(keys.get(1));
+			assertEquals(256, f.length());
+
+			try (FileInputStream fis = new FileInputStream(f)) {
+				for (int i = 0; i < 256 && fis.available() > 0; i++) {
+					assertEquals(expected[32 + i], (byte) fis.read());
+				}
+
+				assertEquals(0, fis.available());
+			}
+		}
+		finally {
+			for (BlobServer s : server) {
+				if (s != null) {
+					s.shutdown();
+				}
+			}
+
+			if (cache != null) {
+				cache.shutdown();
+			}
+
+			if (libCache != null) {
+				libCache.shutdown();
+			}
+		}
+
+		// Verify everything is clean
+		File[] recoveryFiles = recoveryDir.listFiles();
+		assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length);
+	}
+}


[26/47] flink git commit: [FLINK-2804] [runtime] Add blocking job submission support for HA

Posted by tr...@apache.org.
[FLINK-2804] [runtime] Add blocking job submission support for HA

The JobClientActor is now repsonsible for receiving the JobStatus updates from
a newly elected leader. It uses the LeaderRetrievalService to be notified about
new leaders. The actor can only be used to submit a single job to the JM. Once
it received a job from the Client it tries to send it to the current leader.
If no leader is available, a connection timeout is triggered. If the job could
be sent to the JM, a submission timeout is triggered if the JobClientActor does
not receive a JobSubmitSuccess message within the timeout interval. If the
connection to the leader is lost after having submitted a job, a connection
timeout is triggered if the JobClientActor cannot reconnect to another JM within
the timeout interval. The JobClient simply awaits on the completion of the
returned future to the SubmitJobAndWait message.

Added test cases for JobClientActor exceptions

This closes #1249.


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

Branch: refs/heads/master
Commit: d18f5809808221f6bd7c045aa78dfe01dab2afdf
Parents: c3a4d1d
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Oct 8 01:52:07 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:52 2015 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/program/Client.java | 108 +++---
 .../RemoteExecutorHostnameResolutionTest.java   |   6 +-
 .../apache/flink/runtime/client/JobClient.java  |  50 +--
 .../flink/runtime/client/JobClientActor.java    | 360 +++++++++++++++----
 ...obClientActorConnectionTimeoutException.java |  35 ++
 ...obClientActorSubmissionTimeoutException.java |  35 ++
 .../flink/runtime/jobmanager/JobManager.scala   |   2 +
 .../runtime/messages/JobClientMessages.scala    |  26 +-
 .../runtime/messages/JobManagerMessages.scala   |   6 +-
 .../runtime/minicluster/FlinkMiniCluster.scala  |  24 +-
 .../flink/runtime/taskmanager/TaskManager.scala |   7 +-
 .../client/JobClientActorRecoveryITCase.java    | 163 +++++++++
 .../runtime/client/JobClientActorTest.java      | 228 ++++++++++++
 .../PartialConsumePipelinedResultTest.java      |  13 +-
 .../JobManagerLeaderElectionTest.java           |  25 +-
 .../LeaderChangeStateCleanupTest.java           |   9 +-
 .../LeaderElectionRetrievalTestingCluster.java  |  25 +-
 .../TestingLeaderElectionService.java           |   5 +-
 .../TestingLeaderRetrievalService.java          |  16 +
 .../testingUtils/TestingJobManagerLike.scala    |  15 +
 .../TestingJobManagerMessages.scala             |   9 +-
 .../ZooKeeperLeaderElectionITCase.java          | 117 +++---
 .../src/test/resources/log4j-test.properties    |   7 +-
 23 files changed, 1008 insertions(+), 283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
index dfb9c1b..322c73d 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
@@ -24,8 +24,8 @@ import java.net.URL;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Preconditions;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
@@ -47,7 +47,6 @@ import org.apache.flink.runtime.client.JobClient;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
@@ -81,12 +80,15 @@ public class Client {
 	/** The actor system used to communicate with the JobManager */
 	private final ActorSystem actorSystem;
 
-	/** The actor reference to the JobManager */
-	private final ActorGateway jobManagerGateway;
+	/** Configuration of the client */
+	private final Configuration config;
 
-	/** The timeout for communication between the client and the JobManager */
+	/** Timeout for futures */
 	private final FiniteDuration timeout;
-	
+
+	/** Lookup timeout for the job manager retrieval service */
+	private final FiniteDuration lookupTimeout;
+
 	/**
 	 * If != -1, this field specifies the total number of available slots on the cluster
 	 * connected to the client.
@@ -133,6 +135,7 @@ public class Client {
 	 */
 	public Client(Configuration config, int maxSlots) throws IOException {
 
+		this.config = Preconditions.checkNotNull(config);
 		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
 		this.maxSlots = maxSlots;
 
@@ -144,50 +147,8 @@ public class Client {
 			throw new IOException("Could start client actor system.", e);
 		}
 
-		// from here on, we need to make sure the actor system is shut down on error
-		boolean success = false;
-
-		try {
-
-			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(config);
-			this.timeout = AkkaUtils.getTimeout(config);
-
-			LOG.info("Looking up JobManager");
-			LeaderRetrievalService leaderRetrievalService;
-
-			try {
-				leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
-			} catch (Exception e) {
-				throw new IOException("Could not create the leader retrieval service.", e);
-			}
-
-			try {
-				this.jobManagerGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
-						leaderRetrievalService,
-						actorSystem,
-						lookupTimeout);
-			} catch (LeaderRetrievalException e) {
-				throw new IOException("Failed to retrieve JobManager gateway", e);
-			}
-
-			LOG.info("Leading JobManager actor system address is " + this.jobManagerGateway.path());
-
-			LOG.info("JobManager runs at " + this.jobManagerGateway.path());
-
-			LOG.info("Communication between client and JobManager will have a timeout of " + this.timeout);
-			success = true;
-		} finally {
-			if (!success) {
-				try {
-					this.actorSystem.shutdown();
-
-					// wait at most for 30 seconds, to work around an occasional akka problem
-					actorSystem.awaitTermination(new FiniteDuration(30, TimeUnit.SECONDS));
-				} catch (Throwable t) {
-					LOG.error("Shutting down actor system after error caused another error", t);
-				}
-			}
-		}
+		timeout = AkkaUtils.getTimeout(config);
+		lookupTimeout = AkkaUtils.getTimeout(config);
 	}
 	// ------------------------------------------------------------------------
 	//  Startup & Shutdown
@@ -395,21 +356,30 @@ public class Client {
 	}
 
 	public JobExecutionResult runBlocking(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
-		LOG.info("Checking and uploading JAR files");
+		LeaderRetrievalService leaderRetrievalService;
 		try {
-			JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
-		} catch (IOException e) {
-			throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e);
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+		} catch (Exception e) {
+			throw new ProgramInvocationException("Could not create the leader retrieval service.", e);
 		}
+
 		try {
 			this.lastJobID = jobGraph.getJobID();
-			return JobClient.submitJobAndWait(actorSystem, jobManagerGateway, jobGraph, timeout, printStatusDuringExecution, classLoader);
+			return JobClient.submitJobAndWait(actorSystem, leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader);
 		} catch (JobExecutionException e) {
 			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
 		}
 	}
 
 	public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
+		ActorGateway jobManagerGateway;
+
+		try {
+			jobManagerGateway = getJobManagerGateway();
+		} catch (Exception e) {
+			throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e);
+		}
+
 		LOG.info("Checking and uploading JAR files");
 		try {
 			JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
@@ -432,6 +402,8 @@ public class Client {
 	 * @throws Exception In case an error occurred.
 	 */
 	public void cancel(JobID jobId) throws Exception {
+		ActorGateway jobManagerGateway = getJobManagerGateway();
+
 		Future<Object> response;
 		try {
 			response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout);
@@ -471,6 +443,7 @@ public class Client {
 	 * @return A Map containing the accumulator's name and its value.
 	 */
 	public Map<String, Object> getAccumulators(JobID jobID, ClassLoader loader) throws Exception {
+		ActorGateway jobManagerGateway = getJobManagerGateway();
 
 		Future<Object> response;
 		try {
@@ -520,6 +493,8 @@ public class Client {
 		if (jobIds == null) {
 			throw new IllegalArgumentException("The JobIDs must not be null");
 		}
+
+		ActorGateway jobManagerGateway = getJobManagerGateway();
 		
 		for (JobID jid : jobIds) {
 			if (jid != null) {
@@ -572,4 +547,27 @@ public class Client {
 		return job;
 	}
 
+	// ------------------------------------------------------------------------
+	//  Helper methods
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Returns the {@link ActorGateway} of the current job manager leader using
+	 * the {@link LeaderRetrievalService}.
+	 *
+	 * @return ActorGateway of the current job manager leader
+	 * @throws Exception
+	 */
+	private ActorGateway getJobManagerGateway() throws Exception {
+		LOG.info("Looking up JobManager");
+		LeaderRetrievalService leaderRetrievalService;
+
+		leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+
+		return LeaderRetrievalUtils.retrieveLeaderGateway(
+			leaderRetrievalService,
+			actorSystem,
+			lookupTimeout);
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
index 424f72e..fb5200b 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
@@ -21,10 +21,10 @@ package org.apache.flink.client;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.Configuration;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URL;
@@ -51,7 +51,7 @@ public class RemoteExecutorHostnameResolutionTest {
 			exec.executePlan(getProgram());
 			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (IOException e) {
+		catch (ProgramInvocationException e) {
 			// that is what we want!
 			assertTrue(e.getCause() instanceof UnknownHostException);
 		}
@@ -74,7 +74,7 @@ public class RemoteExecutorHostnameResolutionTest {
 			exec.executePlan(getProgram());
 			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (IOException e) {
+		catch (ProgramInvocationException e) {
 			// that is what we want!
 			assertTrue(e.getCause() instanceof UnknownHostException);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index a436881..0105632 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -28,12 +28,12 @@ import akka.util.Timeout;
 
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobClientMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.util.SerializedThrowable;
@@ -52,7 +52,6 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.util.concurrent.TimeoutException;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -85,43 +84,14 @@ public class JobClient {
 	}
 
 	/**
-	 * Extracts the JobManager's Akka URL from the configuration. If localActorSystem is true, then
-	 * the JobClient is executed in the same actor system as the JobManager. Thus, they can
-	 * communicate locally.
-	 *
-	 * @param config Configuration object containing all user provided configuration values
-	 * @return The socket address of the JobManager actor system
-	 */
-	public static InetSocketAddress getJobManagerAddress(Configuration config) throws IOException {
-		String jobManagerAddress = config.getString(
-				ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-
-		int jobManagerRPCPort = config.getInteger(
-				ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-				ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-
-		if (jobManagerAddress == null) {
-			throw new RuntimeException(
-					"JobManager address has not been specified in the configuration.");
-		}
-
-		try {
-			return new InetSocketAddress(
-					InetAddress.getByName(jobManagerAddress), jobManagerRPCPort);
-		}
-		catch (UnknownHostException e) {
-			throw new IOException("Cannot resolve JobManager hostname " + jobManagerAddress, e);
-		}
-	}
-
-	/**
 	 * Sends a [[JobGraph]] to the JobClient actor specified by jobClient which submits it then to
 	 * the JobManager. The method blocks until the job has finished or the JobManager is no longer
 	 * alive. In the former case, the [[SerializedJobExecutionResult]] is returned and in the latter
 	 * case a [[JobExecutionException]] is thrown.
 	 *
 	 * @param actorSystem The actor system that performs the communication.
-	 * @param jobManagerGateway  Gateway to the JobManager that should execute the job.
+	 * @param leaderRetrievalService Leader retrieval service which used to find the current leading
+	 *                               JobManager
 	 * @param jobGraph    JobGraph describing the Flink job
 	 * @param timeout     Timeout for futures
 	 * @param sysoutLogUpdates prints log updates to system out if true
@@ -131,14 +101,14 @@ public class JobClient {
 	 */
 	public static JobExecutionResult submitJobAndWait(
 			ActorSystem actorSystem,
-			ActorGateway jobManagerGateway,
+			LeaderRetrievalService leaderRetrievalService,
 			JobGraph jobGraph,
 			FiniteDuration timeout,
 			boolean sysoutLogUpdates,
 			ClassLoader classLoader) throws JobExecutionException {
 
 		checkNotNull(actorSystem, "The actorSystem must not be null.");
-		checkNotNull(jobManagerGateway, "The jobManagerGateway must not be null.");
+		checkNotNull(leaderRetrievalService, "The jobManagerGateway must not be null.");
 		checkNotNull(jobGraph, "The jobGraph must not be null.");
 		checkNotNull(timeout, "The timeout must not be null.");
 
@@ -146,12 +116,10 @@ public class JobClient {
 		// the JobManager. It forwards the job submission, checks the success/failure responses, logs
 		// update messages, watches for disconnect between client and JobManager, ...
 
-		Props jobClientActorProps = Props.create(
-				JobClientActor.class,
-				jobManagerGateway.actor(),
-				LOG,
-				sysoutLogUpdates,
-				jobManagerGateway.leaderSessionID());
+		Props jobClientActorProps = JobClientActor.createJobClientActorProps(
+			leaderRetrievalService,
+			timeout,
+			sysoutLogUpdates);
 
 		ActorRef jobClientActor = actorSystem.actorOf(jobClientActorProps);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
index bf747c4..d08046b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
@@ -20,45 +20,93 @@ package org.apache.flink.runtime.client;
 
 import akka.actor.ActorRef;
 import akka.actor.PoisonPill;
+import akka.actor.Props;
 import akka.actor.Status;
 import akka.actor.Terminated;
+import akka.dispatch.Futures;
+import akka.dispatch.OnSuccess;
 import com.google.common.base.Preconditions;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;
 import org.apache.flink.runtime.messages.JobClientMessages;
+import org.apache.flink.runtime.messages.JobClientMessages.JobManagerActorRef;
+import org.apache.flink.runtime.messages.JobClientMessages.JobManagerLeaderAddress;
+import org.apache.flink.runtime.messages.JobClientMessages.SubmitJobAndWait;
 import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.slf4j.Logger;
+import org.apache.flink.runtime.util.SerializedThrowable;
+import scala.concurrent.duration.FiniteDuration;
 
+import java.io.IOException;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 
 /**
  * Actor which constitutes the bridge between the non-actor code and the JobManager. The JobClient
  * is used to submit jobs to the JobManager and to request the port of the BlobManager.
  */
-public class JobClientActor extends FlinkUntypedActor {
-	
-	private final ActorRef jobManager;
-	private final Logger logger;
+public class JobClientActor extends FlinkUntypedActor implements LeaderRetrievalListener {
+
+	private final LeaderRetrievalService leaderRetrievalService;
+
+	/** timeout for futures */
+	private final FiniteDuration timeout;
+
+	/** true if status messages shall be printed to sysout */
 	private final boolean sysoutUpdates;
 
+	/** true if a SubmitJobSuccess message has been received */
+	private boolean jobSuccessfullySubmitted = false;
+
+	/** true if a PoisonPill was taken */
+	private boolean terminated = false;
+
+	/** ActorRef to the current leader */
+	private ActorRef jobManager;
+
 	/** leader session ID of the JobManager when this actor was created */
-	private final UUID leaderSessionID;
+	private UUID leaderSessionID;
 
 	/** Actor which submits a job to the JobManager via this actor */
 	private ActorRef submitter;
 
-	public JobClientActor(ActorRef jobManager, Logger logger, boolean sysoutUpdates,
-			UUID leaderSessionID) {
+	/** JobGraph which shall be submitted to the JobManager */
+	private JobGraph jobGraph;
 
-		this.jobManager = Preconditions.checkNotNull(jobManager, "The JobManager ActorRef must not be null.");
-		this.logger = Preconditions.checkNotNull(logger, "The logger must not be null.");
-
-		this.leaderSessionID = leaderSessionID;
+	public JobClientActor(
+			LeaderRetrievalService leaderRetrievalService,
+			FiniteDuration submissionTimeout,
+			boolean sysoutUpdates) {
+		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService);
+		this.timeout = Preconditions.checkNotNull(submissionTimeout);
 		this.sysoutUpdates = sysoutUpdates;
 	}
-	
+
+	@Override
+	public void preStart() {
+		try {
+			leaderRetrievalService.start(this);
+		} catch (Exception e) {
+			LOG.error("Could not start the leader retrieval service.");
+			throw new RuntimeException("Could not start the leader retrieval service.", e);
+		}
+	}
+
+	@Override
+	public void postStop() {
+		try {
+			leaderRetrievalService.stop();
+		} catch (Exception e) {
+			LOG.warn("Could not properly stop the leader retrieval service.");
+		}
+	}
+
 	@Override
 	protected void handleMessage(Object message) {
 		
@@ -66,50 +114,79 @@ public class JobClientActor extends FlinkUntypedActor {
 
 		if (message instanceof ExecutionGraphMessages.ExecutionStateChanged) {
 			logAndPrintMessage(message);
-		}
-		else if (message instanceof ExecutionGraphMessages.JobStatusChanged) {
+		} else if (message instanceof ExecutionGraphMessages.JobStatusChanged) {
 			logAndPrintMessage(message);
 		}
 
+		// ============ JobManager ActorRef resolution ===============
+
+		else if (message instanceof JobManagerLeaderAddress) {
+			JobManagerLeaderAddress msg = (JobManagerLeaderAddress) message;
+
+			disconnectFromJobManager();
+
+			this.leaderSessionID = msg.leaderSessionID();
+
+			if (msg.address() != null) {
+				// Resolve the job manager leader address to obtain an ActorRef
+				AkkaUtils.getActorRefFuture(msg.address(), getContext().system(), timeout)
+					.onSuccess(new OnSuccess<ActorRef>() {
+						@Override
+						public void onSuccess(ActorRef result) throws Throwable {
+							getSelf().tell(decorateMessage(new JobManagerActorRef(result)), ActorRef.noSender());
+						}
+					}, getContext().dispatcher());
+			}
+		} else if (message instanceof JobManagerActorRef) {
+			// Resolved JobManager ActorRef
+			JobManagerActorRef msg = (JobManagerActorRef) message;
+			connectToJobManager(msg.jobManager());
+
+			if (jobGraph != null && !jobSuccessfullySubmitted) {
+				// if we haven't yet submitted the job successfully
+				tryToSubmitJob(jobGraph);
+			}
+		}
+
 		// =========== Job Life Cycle Messages ===============
 		
 		// submit a job to the JobManager
-		else if (message instanceof JobClientMessages.SubmitJobAndWait) {
-			// sanity check that this no job was submitted through this actor before -
-			// it is a one-shot actor after all
-			if (this.submitter == null) {
-				JobGraph jobGraph = ((JobClientMessages.SubmitJobAndWait) message).jobGraph();
-				if (jobGraph == null) {
-					logger.error("Received null JobGraph");
-					sender().tell(
+		else if (message instanceof SubmitJobAndWait) {
+			// only accept SubmitJobWait messages if we're not about to terminate
+			if (!terminated) {
+				// sanity check that this no job was submitted through this actor before -
+				// it is a one-shot actor after all
+				if (this.submitter == null) {
+					jobGraph = ((SubmitJobAndWait) message).jobGraph();
+					if (jobGraph == null) {
+						LOG.error("Received null JobGraph");
+						sender().tell(
 							decorateMessage(new Status.Failure(new Exception("JobGraph is null"))),
 							getSelf());
-				}
-				else {
-					logger.info("Sending message to JobManager {} to submit job {} ({}) and wait for progress",
-							jobManager.path().toString(), jobGraph.getName(), jobGraph.getJobID());
+					} else {
+						LOG.info("Received job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
 
-					this.submitter = getSender();
-					jobManager.tell(
-						decorateMessage(
-							new JobManagerMessages.SubmitJob(
-								jobGraph,
-								ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES)),
-							getSelf());
-					
-					// make sure we notify the sender when the connection got lost
-					getContext().watch(jobManager);
+						this.submitter = getSender();
+
+						// is only successful if we already know the job manager leader
+						tryToSubmitJob(jobGraph);
+					}
+				} else {
+					// repeated submission - tell failure to sender and kill self
+					String msg = "Received repeated 'SubmitJobAndWait'";
+					LOG.error(msg);
+					getSender().tell(
+						decorateMessage(new Status.Failure(new Exception(msg))), ActorRef.noSender());
+
+					terminate();
 				}
-			}
-			else {
-				// repeated submission - tell failure to sender and kill self
-				String msg = "Received repeated 'SubmitJobAndWait'";
-				logger.error(msg);
+			} else {
+				// we're about to receive a PoisonPill because terminated == true
+				String msg = getClass().getName() + " is about to be terminated. Therefore, the " +
+					"job submission cannot be executed.";
+				LOG.error(msg);
 				getSender().tell(
 					decorateMessage(new Status.Failure(new Exception(msg))), ActorRef.noSender());
-
-				getContext().unwatch(jobManager);
-				getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender());
 			}
 		}
 		// acknowledgement to submit job is only logged, our original
@@ -117,41 +194,80 @@ public class JobClientActor extends FlinkUntypedActor {
 		else if (message instanceof JobManagerMessages.JobResultSuccess ||
 				message instanceof JobManagerMessages.JobResultFailure) {
 			
-			if (logger.isDebugEnabled()) {
-				logger.debug("Received {} message from JobManager", message.getClass().getSimpleName());
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Received {} message from JobManager", message.getClass().getSimpleName());
 			}
 
 			// forward the success to the original job submitter
-			if (this.submitter != null) {
+			if (hasJobBeenSubmitted()) {
 				this.submitter.tell(decorateMessage(message), getSelf());
 			}
-			
-			// we are done, stop ourselves
-			getContext().unwatch(jobManager);
-			getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender());
+
+			terminate();
 		}
 		else if (message instanceof JobManagerMessages.JobSubmitSuccess) {
 			// job was successfully submitted :-)
-			logger.info("Job was successfully submitted to the JobManager");
+			LOG.info("Job was successfully submitted to the JobManager {}.", getSender().path());
+			jobSuccessfullySubmitted = true;
 		}
 
-		// =========== Actor / Communication Failure ===============
+		// =========== Actor / Communication Failure / Timeouts ===============
 		
 		else if (message instanceof Terminated) {
 			ActorRef target = ((Terminated) message).getActor();
 			if (jobManager.equals(target)) {
-				String msg = "Lost connection to JobManager " + jobManager.path();
-				logger.info(msg);
-				submitter.tell(decorateMessage(new Status.Failure(new Exception(msg))), getSelf());
+				LOG.info("Lost connection to JobManager {}. Triggering connection timeout.",
+					jobManager.path());
+				disconnectFromJobManager();
+
+				// we only issue a connection timeout if we have submitted a job before
+				// otherwise, we might have some more time to find another job manager
+				// Important: The ConnectionTimeout message is filtered out in case that we are
+				// notified about a new leader by setting the new leader session ID, because
+				// ConnectionTimeout extends RequiresLeaderSessionID
+				if (hasJobBeenSubmitted()) {
+					getContext().system().scheduler().scheduleOnce(
+						timeout,
+						getSelf(),
+						decorateMessage(JobClientMessages.getConnectionTimeout()),
+						getContext().dispatcher(),
+						ActorRef.noSender());
+				}
 			} else {
-				logger.error("Received 'Terminated' for unknown actor " + target);
+				LOG.warn("Received 'Terminated' for unknown actor " + target);
+			}
+		} else if (JobClientMessages.getConnectionTimeout().equals(message)) {
+			// check if we haven't found a job manager yet
+			if (!isConnected()) {
+				if (hasJobBeenSubmitted()) {
+					submitter.tell(
+						decorateMessage(new Status.Failure(
+							new JobClientActorConnectionTimeoutException("Lost connection to the JobManager."))),
+						getSelf());
+				}
+				// Connection timeout reached, let's terminate
+				terminate();
+			}
+		} else if (JobClientMessages.getSubmissionTimeout().equals(message)) {
+			// check if our job submission was successful in the meantime
+			if (!jobSuccessfullySubmitted) {
+				if (hasJobBeenSubmitted()) {
+					submitter.tell(
+						decorateMessage(new Status.Failure(
+							new JobClientActorSubmissionTimeoutException("Job submission to the JobManager timed out."))),
+						getSelf());
+				}
+
+				// We haven't heard back from the job manager after sending the job graph to him,
+				// therefore terminate
+				terminate();
 			}
 		}
 
 		// =========== Unknown Messages ===============
 		
 		else {
-			logger.error("JobClient received unknown message: " + message);
+			LOG.error("JobClient received unknown message: " + message);
 		}
 	}
 
@@ -161,9 +277,133 @@ public class JobClientActor extends FlinkUntypedActor {
 	}
 
 	private void logAndPrintMessage(Object message) {
-		logger.info(message.toString());
+		LOG.info(message.toString());
 		if (sysoutUpdates) {
 			System.out.println(message.toString());
 		}
 	}
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		getSelf().tell(
+			decorateMessage(new JobManagerLeaderAddress(leaderAddress, leaderSessionID)),
+			getSelf());
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		LOG.error("Error occurred in the LeaderRetrievalService.", exception);
+		getSelf().tell(decorateMessage(PoisonPill.getInstance()), getSelf());
+	}
+
+	private void disconnectFromJobManager() {
+		if (jobManager != ActorRef.noSender()) {
+			getContext().unwatch(jobManager);
+			jobManager = ActorRef.noSender();
+		}
+	}
+
+	private void connectToJobManager(ActorRef jobManager) {
+		if (jobManager != ActorRef.noSender()) {
+			getContext().unwatch(jobManager);
+		}
+
+		LOG.info("Connected to new JobManager {}.", jobManager.path());
+
+		this.jobManager = jobManager;
+		getContext().watch(jobManager);
+	}
+
+	private void tryToSubmitJob(final JobGraph jobGraph) {
+		this.jobGraph = jobGraph;
+
+		if (isConnected()) {
+			LOG.info("Sending message to JobManager {} to submit job {} ({}) and wait for progress",
+				jobManager.path().toString(), jobGraph.getName(), jobGraph.getJobID());
+
+			Futures.future(new Callable<Object>() {
+				@Override
+				public Object call() throws Exception {
+					ActorGateway jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID);
+
+					LOG.info("Upload jar files to job manager {}.", jobManager.path());
+
+					try {
+						JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
+					} catch (IOException exception) {
+						getSelf().tell(
+							decorateMessage(new JobManagerMessages.JobResultFailure(
+								new SerializedThrowable(
+									new JobSubmissionException(
+										jobGraph.getJobID(),
+										"Could not upload the jar files to the job manager.",
+										exception)
+								)
+							)),
+							ActorRef.noSender()
+						);
+					}
+
+					LOG.info("Submit job to the job manager {}.", jobManager.path());
+
+					jobManager.tell(
+						decorateMessage(
+							new JobManagerMessages.SubmitJob(
+								jobGraph,
+								ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES)),
+						getSelf());
+
+					// issue a SubmissionTimeout message to check that we submit the job within
+					// the given timeout
+					getContext().system().scheduler().scheduleOnce(
+						timeout,
+						getSelf(),
+						decorateMessage(JobClientMessages.getSubmissionTimeout()),
+						getContext().dispatcher(),
+						ActorRef.noSender());
+
+					return null;
+				}
+			}, getContext().dispatcher());
+		} else {
+			LOG.info("Could not submit job {} ({}), because there is no connection to a " +
+					"JobManager.",
+				jobGraph.getName(), jobGraph.getJobID());
+
+			// We want to submit a job, but we haven't found a job manager yet.
+			// Let's give him another chance to find a job manager within the given timeout.
+			getContext().system().scheduler().scheduleOnce(
+				timeout,
+				getSelf(),
+				decorateMessage(JobClientMessages.getConnectionTimeout()),
+				getContext().dispatcher(),
+				ActorRef.noSender()
+			);
+		}
+	}
+
+	private void terminate() {
+		terminated = true;
+		disconnectFromJobManager();
+		getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender());
+	}
+
+	private boolean isConnected() {
+		return jobManager != ActorRef.noSender();
+	}
+
+	private boolean hasJobBeenSubmitted() {
+		return submitter != ActorRef.noSender();
+	}
+
+	public static Props createJobClientActorProps(
+			LeaderRetrievalService leaderRetrievalService,
+			FiniteDuration timeout,
+			boolean sysoutUpdates) {
+		return Props.create(
+			JobClientActor.class,
+			leaderRetrievalService,
+			timeout,
+			sysoutUpdates);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java
new file mode 100644
index 0000000..72a5658
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.client;
+
+/**
+ * Exception which is thrown when the {@link JobClientActor} wants to submit a job to
+ * the job manager but has not found one after a given timeout interval.
+ */
+public class JobClientActorConnectionTimeoutException extends Exception {
+	private static final long serialVersionUID = 2287747430528388637L;
+
+	public JobClientActorConnectionTimeoutException(String msg) {
+		super(msg);
+	}
+
+	public JobClientActorConnectionTimeoutException(String msg, Throwable cause) {
+		super(msg, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java
new file mode 100644
index 0000000..2d39462
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.client;
+
+/**
+ * Exception which is thrown by the {@link JobClientActor} if it has not heard back from the job
+ * manager after it has submitted a job to it within a given timeout interval.
+ */
+public class JobClientActorSubmissionTimeoutException extends Exception {
+	private static final long serialVersionUID = 8762463142030454853L;
+
+	public JobClientActorSubmissionTimeoutException(String msg) {
+		super(msg);
+	}
+
+	public JobClientActorSubmissionTimeoutException(String msg, Throwable cause) {
+		super(msg, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/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 eef28d8..055274d 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
@@ -370,6 +370,8 @@ class JobManager(
         akka.serialization.JavaSerializer.currentSystem.withValue(
           context.system.asInstanceOf[ExtendedActorSystem]) {
 
+          log.info(s"Recovering all jobs.")
+
           val jobGraphs = submittedJobGraphs.recoverJobGraphs().asScala
 
           if (!leaderElectionService.hasLeadership()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala
index ac37493..a60fa7a 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.messages
 
+import java.util.UUID
+
+import akka.actor.ActorRef
 import org.apache.flink.runtime.jobgraph.JobGraph
-import org.apache.flink.runtime.util.SerializedThrowable
 
 /**
  * This object contains the [[org.apache.flink.runtime.client.JobClient]] specific messages
@@ -47,4 +49,26 @@ object JobClientMessages {
    * @param jobGraph The job to be executed.
    */
   case class SubmitJobDetached(jobGraph: JobGraph)
+
+  /** Notifies the JobClientActor about a new leader address and a leader session ID.
+    *
+    * @param address New leader address
+    * @param leaderSessionID New leader session ID
+    */
+  case class JobManagerLeaderAddress(address: String, leaderSessionID: UUID)
+
+  /** Notifies the JobClientActor about the ActorRef of the new leader.
+    *
+    * @param jobManager ActorRef of the new leader
+    */
+  case class JobManagerActorRef(jobManager: ActorRef) extends RequiresLeaderSessionID
+
+  /** Message which is triggered when the submission timeout has been reached. */
+  case object SubmissionTimeout extends RequiresLeaderSessionID
+
+  /** Messaeg which is triggered when the connection timeout has been reached. */
+  case object ConnectionTimeout extends RequiresLeaderSessionID
+
+  def getSubmissionTimeout(): AnyRef = SubmissionTimeout
+  def getConnectionTimeout(): AnyRef = ConnectionTimeout
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index 8097bdc..8a4d27b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -84,7 +84,7 @@ object JobManagerMessages {
   /**
    * Triggers recovery of all available jobs.
    */
-  case class RecoverAllJobs() extends RequiresLeaderSessionID
+  case object RecoverAllJobs extends RequiresLeaderSessionID
 
   /**
    * Cancels a job with the given [[jobID]] at the JobManager. The result of the cancellation is
@@ -427,4 +427,8 @@ object JobManagerMessages {
   def getRequestArchive: AnyRef = {
     RequestArchive
   }
+
+  def getRecoverAllJobs: AnyRef = {
+    RecoverAllJobs
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index 2df3437..9c4381e 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -38,7 +38,7 @@ import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode}
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalService, LeaderRetrievalListener,
 StandaloneLeaderRetrievalService}
 import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtAnyJobManager
-import org.apache.flink.runtime.util.ZooKeeperUtils
+import org.apache.flink.runtime.util.{LeaderRetrievalUtils, StandaloneUtils, ZooKeeperUtils}
 import org.apache.flink.runtime.webmonitor.{WebMonitorUtils, WebMonitor}
 
 import org.slf4j.LoggerFactory
@@ -388,29 +388,29 @@ abstract class FlinkMiniCluster(
     : JobExecutionResult = {
     submitJobAndWait(jobGraph, printUpdates, timeout)
   }
+
+  def submitJobAndWait(
+    jobGraph: JobGraph,
+    printUpdates: Boolean,
+    timeout: FiniteDuration)
+  : JobExecutionResult = {
+    submitJobAndWait(jobGraph, printUpdates, timeout, createLeaderRetrievalService())
+  }
   
   @throws(classOf[JobExecutionException])
   def submitJobAndWait(
       jobGraph: JobGraph,
       printUpdates: Boolean,
-      timeout: FiniteDuration)
+      timeout: FiniteDuration,
+      leaderRetrievalService: LeaderRetrievalService)
     : JobExecutionResult = {
 
     val clientActorSystem = startJobClientActorSystem(jobGraph.getJobID)
 
      try {
-       val jobManagerGateway = try {
-           getLeaderGateway(timeout)
-         } catch {
-           case e: Exception => throw new JobExecutionException(
-             jobGraph.getJobID,
-             "Could not retrieve leading job manager gateway.",
-             e)
-         }
-
      JobClient.submitJobAndWait(
        clientActorSystem,
-       jobManagerGateway,
+       leaderRetrievalService,
        jobGraph,
        timeout,
        printUpdates,

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index b28fb73..d9d9596 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -1117,7 +1117,12 @@ class TaskManager(
 
     currentJobManager match {
       case Some(jm) =>
-        handleJobManagerDisconnect(jm, s"JobManager ${newJobManagerAkkaURL} was elected as leader.")
+        Option(newJobManagerAkkaURL) match {
+          case Some(newJMAkkaURL) =>
+            handleJobManagerDisconnect(jm, s"JobManager ${newJMAkkaURL} was elected as leader.")
+          case None =>
+            handleJobManagerDisconnect(jm, s"Old JobManager lost its leadership.")
+        }
       case None =>
     }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java
new file mode 100644
index 0000000..a93a515
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java
@@ -0,0 +1,163 @@
+/*
+ * 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.client;
+
+import akka.actor.PoisonPill;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.concurrent.Await;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+
+public class JobClientActorRecoveryITCase extends TestLogger {
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public static TestingServer zkServer;
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		zkServer = new TestingServer();
+
+		zkServer.start();
+	}
+
+	public static void teardown() throws Exception {
+		if (zkServer != null) {
+			zkServer.stop();
+			zkServer = null;
+		}
+	}
+
+	/**
+	 * Tests wether the JobClientActor can connect to a newly elected leading job manager to obtain
+	 * the JobExecutionResult. The submitted job blocks for the first execution attempt. The
+	 * leading job manager will be killed so that the second job manager will be elected as the
+	 * leader. The newly elected leader has to retrieve the checkpointed job from ZooKeeper
+	 * and continue its execution. This time, the job does not block and, thus, can be finished.
+	 * The execution result should be sent to the JobClientActor which originally submitted the
+	 * job.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testJobClientRecovery() throws Exception {
+		File rootFolder = tempFolder.getRoot();
+
+		Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
+			zkServer.getConnectString(),
+			rootFolder.getPath());
+
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 2);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+
+		final TestingCluster cluster = new TestingCluster(config);
+		cluster.start();
+
+		JobVertex blockingVertex = new JobVertex("Blocking Vertex");
+		blockingVertex.setInvokableClass(BlockingTask.class);
+		blockingVertex.setParallelism(1);
+		final JobGraph jobGraph = new JobGraph("Blocking Test Job", blockingVertex);
+		final Promise<JobExecutionResult> promise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+
+		Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow();
+
+		try {
+			Thread submitter = new Thread(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						JobExecutionResult result = cluster.submitJobAndWait(jobGraph, false);
+						promise.success(result);
+					} catch (Exception e) {
+						promise.failure(e);
+					}
+				}
+			});
+
+			submitter.start();
+
+			synchronized (BlockingTask.waitLock) {
+				while (BlockingTask.HasBlockedExecution < 1 && deadline.hasTimeLeft()) {
+					BlockingTask.waitLock.wait(deadline.timeLeft().toMillis());
+				}
+			}
+
+			if (deadline.isOverdue()) {
+				Assert.fail("The job has not blocked within the given deadline.");
+			}
+
+			ActorGateway gateway = cluster.getLeaderGateway(deadline.timeLeft());
+
+			gateway.tell(TestingJobManagerMessages.getDisablePostStop());
+			gateway.tell(PoisonPill.getInstance());
+
+			// if the job fails then an exception is thrown here
+			Await.result(promise.future(), deadline.timeLeft());
+		} finally {
+			cluster.shutdown();
+		}
+	}
+
+	public static class BlockingTask extends AbstractInvokable {
+
+		private volatile static int BlockExecution = 1;
+		private volatile static int HasBlockedExecution = 0;
+		private static Object waitLock = new Object();
+
+		@Override
+		public void registerInputOutput() throws Exception {
+			// Nothing to do
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			if (BlockExecution > 0) {
+				BlockExecution--;
+
+				// Tell the test that it's OK to kill the leader
+				synchronized (waitLock) {
+					HasBlockedExecution++;
+					waitLock.notifyAll();
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
new file mode 100644
index 0000000..00ad632
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.client;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
+import akka.actor.Props;
+import akka.pattern.Patterns;
+import akka.testkit.JavaTestKit;
+import akka.util.Timeout;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.FlinkUntypedActor;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.messages.JobClientMessages;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+public class JobClientActorTest extends TestLogger {
+
+	private static ActorSystem system;
+	private static JobGraph testJobGraph = new JobGraph("Test Job");
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createLocalActorSystem(new Configuration());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+		system = null;
+	}
+
+	/** Tests that a {@link JobClientActorSubmissionTimeoutException} is thrown when the job cannot
+	 * be submitted by the JobClientActor. This is here the case, because the started JobManager
+	 * never replies to a SubmitJob message.
+	 *
+	 * @throws Exception
+	 */
+	@Test(expected=JobClientActorSubmissionTimeoutException.class)
+	public void testSubmissionTimeout() throws Exception {
+		FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS);
+		FiniteDuration timeout = jobClientActorTimeout.$times(2);
+
+		UUID leaderSessionID = UUID.randomUUID();
+
+		ActorRef jobManager = system.actorOf(
+			Props.create(
+				PlainActor.class,
+				leaderSessionID));
+
+		TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService(
+			jobManager.path().toString(),
+			leaderSessionID
+		);
+
+		Props jobClientActorProps = JobClientActor.createJobClientActorProps(
+			testingLeaderRetrievalService,
+			jobClientActorTimeout,
+			false);
+
+		ActorRef jobClientActor = system.actorOf(jobClientActorProps);
+
+
+		Future<Object> jobExecutionResult = Patterns.ask(
+			jobClientActor,
+			new JobClientMessages.SubmitJobAndWait(testJobGraph),
+			new Timeout(timeout));
+
+		Await.result(jobExecutionResult, timeout);
+	}
+
+	/** Tests that a {@link org.apache.flink.runtime.client.JobClientActorConnectionTimeoutException}
+	 * is thrown when the JobClientActor wants to submit a job but has not connected to a JobManager.
+	 *
+	 * @throws Exception
+	 */
+	@Test(expected=JobClientActorConnectionTimeoutException.class)
+	public void testConnectionTimeoutWithoutJobManager() throws Exception {
+		FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS);
+		FiniteDuration timeout = jobClientActorTimeout.$times(2);
+
+		TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService();
+
+		Props jobClientActorProps = JobClientActor.createJobClientActorProps(
+			testingLeaderRetrievalService,
+			jobClientActorTimeout,
+			false);
+
+		ActorRef jobClientActor = system.actorOf(jobClientActorProps);
+
+		Future<Object> jobExecutionResult = Patterns.ask(
+			jobClientActor,
+			new JobClientMessages.SubmitJobAndWait(testJobGraph),
+			new Timeout(timeout));
+
+		Await.result(jobExecutionResult, timeout);
+	}
+
+	/** Tests that a {@link org.apache.flink.runtime.client.JobClientActorConnectionTimeoutException}
+	 * is thrown after a successful job submission if the JobManager dies.
+	 *
+	 * @throws Exception
+	 */
+	@Test(expected=JobClientActorConnectionTimeoutException.class)
+	public void testConnectionTimeoutAfterJobSubmission() throws Exception {
+		FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS);
+		FiniteDuration timeout = jobClientActorTimeout.$times(2);
+
+		UUID leaderSessionID = UUID.randomUUID();
+
+		ActorRef jobManager = system.actorOf(
+			Props.create(
+				JobAcceptingActor.class,
+				leaderSessionID));
+
+		TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService(
+			jobManager.path().toString(),
+			leaderSessionID
+		);
+
+		Props jobClientActorProps = JobClientActor.createJobClientActorProps(
+			testingLeaderRetrievalService,
+			jobClientActorTimeout,
+			false);
+
+		ActorRef jobClientActor = system.actorOf(jobClientActorProps);
+
+		Future<Object> jobExecutionResult = Patterns.ask(
+			jobClientActor,
+			new JobClientMessages.SubmitJobAndWait(testJobGraph),
+			new Timeout(timeout));
+
+		Future<Object> waitFuture = Patterns.ask(jobManager, new RegisterTest(), new Timeout(timeout));
+
+		Await.result(waitFuture, timeout);
+
+		jobManager.tell(PoisonPill.getInstance(), ActorRef.noSender());
+
+		Await.result(jobExecutionResult, timeout);
+	}
+
+	public static class PlainActor extends FlinkUntypedActor {
+
+		private final UUID leaderSessionID;
+
+		public PlainActor(UUID leaderSessionID) {
+			this.leaderSessionID = leaderSessionID;
+		}
+
+		@Override
+		protected void handleMessage(Object message) throws Exception {
+
+		}
+
+		@Override
+		protected UUID getLeaderSessionID() {
+			return leaderSessionID;
+		}
+	}
+
+	public static class JobAcceptingActor extends FlinkUntypedActor {
+		private final UUID leaderSessionID;
+		private boolean jobAccepted = false;
+		private ActorRef testFuture = ActorRef.noSender();
+
+		public JobAcceptingActor(UUID leaderSessionID) {
+			this.leaderSessionID = leaderSessionID;
+		}
+
+		@Override
+		protected void handleMessage(Object message) throws Exception {
+			if (message instanceof JobManagerMessages.SubmitJob) {
+				getSender().tell(
+					new JobManagerMessages.JobSubmitSuccess(((JobManagerMessages.SubmitJob) message).jobGraph().getJobID()),
+					getSelf());
+
+				jobAccepted = true;
+
+				if(testFuture != ActorRef.noSender()) {
+					testFuture.tell(Messages.getAcknowledge(), getSelf());
+				}
+			} else if (message instanceof RegisterTest) {
+				testFuture = getSender();
+
+				if (jobAccepted) {
+					testFuture.tell(Messages.getAcknowledge(), getSelf());
+				}
+			}
+		}
+
+		@Override
+		protected UUID getLeaderSessionID() {
+			return leaderSessionID;
+		}
+	}
+
+	public static class RegisterTest{}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
index 5753cde..aa03fe1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
@@ -18,10 +18,8 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
-import akka.actor.ActorSystem;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobClient;
 import org.apache.flink.runtime.io.network.api.reader.BufferReader;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -46,7 +44,6 @@ public class PartialConsumePipelinedResultTest {
 	private final static int NUMBER_OF_NETWORK_BUFFERS = 128;
 
 	private static TestingCluster flink;
-	private static ActorSystem jobClient;
 
 	@BeforeClass
 	public static void setUp() throws Exception {
@@ -59,8 +56,6 @@ public class PartialConsumePipelinedResultTest {
 		flink = new TestingCluster(config, true);
 
 		flink.start();
-
-		jobClient = JobClient.startJobClientActorSystem(flink.configuration());
 	}
 
 	@AfterClass
@@ -102,13 +97,7 @@ public class PartialConsumePipelinedResultTest {
 		sender.setSlotSharingGroup(slotSharingGroup);
 		receiver.setSlotSharingGroup(slotSharingGroup);
 
-		JobClient.submitJobAndWait(
-				jobClient,
-				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
-				jobGraph,
-				TestingUtils.TESTING_DURATION(),
-				false,
-				this.getClass().getClassLoader());
+		flink.submitJobAndWait(jobGraph, false, TestingUtils.TESTING_DURATION());
 	}
 
 	// ---------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
index bbd8fad..c804830 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
@@ -43,11 +43,14 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.testingUtils.TestingJobManager;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -56,6 +59,9 @@ import java.util.concurrent.TimeUnit;
 
 public class JobManagerLeaderElectionTest extends TestLogger {
 
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
 	private static ActorSystem actorSystem;
 	private static TestingServer testingServer;
 	private static Timeout timeout = new Timeout(TestingUtils.TESTING_DURATION());
@@ -84,12 +90,10 @@ public class JobManagerLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testLeaderElection() throws Exception {
-		final Configuration configuration = new Configuration();
-
-		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
-		configuration.setString(
-			ConfigConstants.ZOOKEEPER_QUORUM_KEY,
-			testingServer.getConnectString());
+		final Configuration configuration = ZooKeeperTestUtils
+			.createZooKeeperRecoveryModeConfig(
+				testingServer.getConnectString(),
+				tempFolder.getRoot().getPath());
 
 		ActorRef jm = null;
 
@@ -115,12 +119,11 @@ public class JobManagerLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testLeaderReelection() throws Exception {
-		final Configuration configuration = new Configuration();
+		final Configuration configuration = ZooKeeperTestUtils
+			.createZooKeeperRecoveryModeConfig(
+				testingServer.getConnectString(),
+				tempFolder.getRoot().getPath());
 
-		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
-		configuration.setString(
-			ConfigConstants.ZOOKEEPER_QUORUM_KEY,
-			testingServer.getConnectString());
 
 		ActorRef jm;
 		ActorRef jm2 = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
index 8dd380e..0b84474 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
@@ -129,7 +129,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 
 		// try to resubmit now the non-blocking job, it should complete successfully
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
-		cluster.submitJobAndWait(job, false, timeout);
+		cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(jm2.path(), jm2.leaderSessionID()));
 	}
 
 	/**
@@ -207,7 +207,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		UUID leaderSessionID = UUID.randomUUID();
 		UUID newLeaderSessionID = UUID.randomUUID();
 
-		FiniteDuration shortTimeout = new FiniteDuration(20, TimeUnit.SECONDS);
+		FiniteDuration shortTimeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
 		cluster.grantLeadership(0, leaderSessionID);
 		cluster.notifyRetrievalListeners(0, leaderSessionID);
@@ -244,10 +244,11 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 
 		cluster.waitForTaskManagersToBeRegistered();
 
+		ActorGateway leaderGateway = cluster.getLeaderGateway(timeout);
+
 		// try to resubmit now the non-blocking job, it should complete successfully
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
-		cluster.submitJobAndWait(job, false, timeout);
-
+		cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(leaderGateway.path(), leaderGateway.leaderSessionID()));
 	}
 
 	public JobGraph createBlockingJob(int parallelism) {

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
index 5b63107..c83f548 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
@@ -25,6 +25,8 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import scala.Option;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -39,11 +41,8 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
 	private final boolean useSingleActorSystem;
 	private final StreamingMode streamingMode;
 
-	public TestingLeaderElectionService[] leaderElectionServices;
-	public TestingLeaderRetrievalService[] leaderRetrievalServices;
-
-	private int leaderElectionServiceCounter = 0;
-	private int leaderRetrievalServiceCounter = 0;
+	public List<TestingLeaderElectionService> leaderElectionServices;
+	public List<TestingLeaderRetrievalService> leaderRetrievalServices;
 
 	private int leaderIndex = -1;
 
@@ -58,8 +57,8 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
 		this.useSingleActorSystem = singleActorSystem;
 		this.streamingMode = streamingMode;
 
-		leaderElectionServices = new TestingLeaderElectionService[this.numJobManagers()];
-		leaderRetrievalServices = new TestingLeaderRetrievalService[this.numTaskManagers() + 1];
+		leaderElectionServices = new ArrayList<TestingLeaderElectionService>();
+		leaderRetrievalServices = new ArrayList<TestingLeaderRetrievalService>();
 	}
 
 	@Override
@@ -79,18 +78,18 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
 
 	@Override
 	public Option<LeaderElectionService> createLeaderElectionService() {
-		leaderElectionServices[leaderElectionServiceCounter] = new TestingLeaderElectionService();
+		leaderElectionServices.add(new TestingLeaderElectionService());
 
-		LeaderElectionService result = leaderElectionServices[leaderElectionServiceCounter++];
+		LeaderElectionService result = leaderElectionServices.get(leaderElectionServices.size() - 1);
 
 		return Option.apply(result);
 	}
 
 	@Override
 	public LeaderRetrievalService createLeaderRetrievalService() {
-		leaderRetrievalServices[leaderRetrievalServiceCounter] = new TestingLeaderRetrievalService();
+		leaderRetrievalServices.add(new TestingLeaderRetrievalService());
 
-		return leaderRetrievalServices[leaderRetrievalServiceCounter++];
+		return leaderRetrievalServices.get(leaderRetrievalServices.size() - 1);
 	}
 
 	@Override
@@ -103,11 +102,11 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
 	public void grantLeadership(int index, UUID leaderSessionID) {
 		if(leaderIndex >= 0) {
 			// first revoke leadership
-			leaderElectionServices[leaderIndex].notLeader();
+			leaderElectionServices.get(leaderIndex).notLeader();
 		}
 
 		// make the JM with index the new leader
-		leaderElectionServices[index].isLeader(leaderSessionID);
+		leaderElectionServices.get(index).isLeader(leaderSessionID);
 
 		leaderIndex = index;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
index ea058f4..4e119fe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
@@ -18,16 +18,13 @@
 
 package org.apache.flink.runtime.leaderelection;
 
-import java.io.Serializable;
 import java.util.UUID;
 
 /**
  * Test {@link LeaderElectionService} implementation which directly forwards isLeader and notLeader
  * calls to the contender.
  */
-public class TestingLeaderElectionService implements LeaderElectionService, Serializable {
-
-	private static final long serialVersionUID = -8007939683948014574L;
+public class TestingLeaderElectionService implements LeaderElectionService {
 
 	private LeaderContender contender;
 	private boolean hasLeadership = false;

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
index 43902fd..c44fc2a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
@@ -29,11 +29,27 @@ import java.util.UUID;
  */
 public class TestingLeaderRetrievalService implements LeaderRetrievalService {
 
+	private final String leaderAddress;
+	private final UUID leaderSessionID;
+
 	private LeaderRetrievalListener listener;
 
+	public TestingLeaderRetrievalService() {
+		this(null, null);
+	}
+
+	public TestingLeaderRetrievalService(String leaderAddress, UUID leaderSessionID) {
+		this.leaderAddress = leaderAddress;
+		this.leaderSessionID = leaderSessionID;
+	}
+
 	@Override
 	public void start(LeaderRetrievalListener listener) throws Exception {
 		this.listener = listener;
+
+		if (leaderAddress != null) {
+			listener.notifyLeaderAddress(leaderAddress, leaderSessionID);
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
index e91f068..b8f4ede 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
@@ -70,6 +70,18 @@ trait TestingJobManagerLike extends FlinkActor {
 
   var disconnectDisabled = false
 
+  var postStopEnabled = true
+
+  abstract override def postStop(): Unit = {
+    if (postStopEnabled) {
+      super.postStop()
+    } else {
+      // only stop leader election service to revoke the leadership of this JM so that a new JM
+      // can be elected leader
+      leaderElectionService.stop()
+    }
+  }
+
   abstract override def handleMessage: Receive = {
     handleTestingMessage orElse super.handleMessage
   }
@@ -270,6 +282,9 @@ trait TestingJobManagerLike extends FlinkActor {
     case DisableDisconnect =>
       disconnectDisabled = true
 
+    case DisablePostStop =>
+      postStopEnabled = false
+
     case msg: Disconnect =>
       if (!disconnectDisabled) {
         super.handleMessage(msg)

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
index 4f5cf14..e4d0a6f 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
@@ -87,5 +87,12 @@ object TestingJobManagerMessages {
    */
   case class NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager: Int)
 
-  def getNotifyWhenLeader: AnyRef = NotifyWhenLeader
+  /** Disables the post stop method of the [[TestingJobManager]].
+    *
+    * Only the leaderElectionService is stopped in the postStop method call to revoke the leadership
+    */
+  case object DisablePostStop
+
+  def getNotifyWhenLeader(): AnyRef = NotifyWhenLeader
+  def getDisablePostStop(): AnyRef = DisablePostStop
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
index ed2113a..5c7a932 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -22,19 +22,20 @@ import akka.actor.ActorSystem;
 import akka.actor.Kill;
 import akka.actor.PoisonPill;
 import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobClient;
-import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
@@ -42,9 +43,13 @@ import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
+import scala.concurrent.impl.Promise;
 
 import java.io.File;
 import java.io.IOException;
@@ -128,7 +133,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 	@Test
 	public void testJobExecutionOnClusterWithLeaderReelection() throws Exception {
 		int numJMs = 10;
-		int numTMs = 3;
+		int numTMs = 2;
 		int numSlotsPerTM = 3;
 		int parallelism = numTMs * numSlotsPerTM;
 
@@ -141,10 +146,9 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
 		configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath());
 
-		// @TODO @tillrohrmann temporary "disable" recovery, because currently the client does
-		// not need to resubmit a failed job to a new leader. Should we keep this test and
-		// disable recovery fully or will this be subsumed by the real client changes anyways?
-		configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, timeout.toString());
+		// we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make
+		// sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message
+		configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, AkkaUtils.INF_TIMEOUT().toString());
 
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
 
@@ -186,51 +190,55 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 			thread.start();
 
+			Deadline deadline = timeout.$times(3).fromNow();
+
 			// Kill all JobManager except for two
-			for (int i = 0; i < numJMs - 2; i++) {
-				ActorGateway jm = cluster.getLeaderGateway(timeout);
+			for (int i = 0; i < numJMs; i++) {
+				ActorGateway jm = cluster.getLeaderGateway(deadline.timeLeft());
 
 				cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor());
 
-				Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout);
-
-				Await.ready(future, timeout);
+				// recover all jobs, sent manually
+				log.info("Sent recover all jobs manually to job manager {}.", jm.path());
+				jm.tell(JobManagerMessages.getRecoverAllJobs());
 
-				cluster.clearLeader();
-
-				jm.tell(Kill.getInstance());
-			}
+				if (i < numJMs - 1) {
+					Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), deadline.timeLeft());
 
-			ActorGateway jm = cluster.getLeaderGateway(timeout);
+					Await.ready(future, deadline.timeLeft());
 
-			cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor());
+					cluster.clearLeader();
 
-			Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout);
+					if (i == numJMs - 2) {
+						Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
+					}
 
-			Await.ready(future, timeout);
+					log.info("Kill job manager {}.", jm.path());
 
-			cluster.clearLeader();
+					jm.tell(TestingJobManagerMessages.getDisablePostStop());
+					jm.tell(Kill.getInstance());
+				}
+			}
 
-			// set the BlockinOnceReceiver for the execution on the last JM to non-blocking, so
-			// that it can succeed
-			Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
+			log.info("Waiting for submitter thread to terminate.");
 
-			jm.tell(PoisonPill.getInstance());
+			thread.join(deadline.timeLeft().toMillis());
 
-			thread.join(timeout.toMillis());
+			log.info("Submitter thread has terminated.");
 
 			if (thread.isAlive()) {
-				jobSubmission.finished = true;
 				fail("The job submission thread did not stop (meaning it did not succeeded in" +
 						"executing the test job.");
 			}
+
+			Await.result(jobSubmission.resultPromise.future(), deadline.timeLeft());
 		}
 		finally {
 			if (clientActorSystem != null) {
 				cluster.shutdownJobClientActorSystem(clientActorSystem);
 			}
 
-			if (thread != null && thread.isAlive() && jobSubmission != null) {
+			if (thread != null && thread.isAlive()) {
 				jobSubmission.finished = true;
 			}
 			cluster.stop();
@@ -238,12 +246,15 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 	}
 
 	public static class JobSubmitterRunnable implements Runnable {
+		private static final Logger LOG = LoggerFactory.getLogger(JobSubmitterRunnable.class);
 		boolean finished = false;
 
 		final ActorSystem clientActorSystem;
 		final ForkableFlinkMiniCluster cluster;
 		final JobGraph graph;
 
+		final Promise<JobExecutionResult> resultPromise = new Promise.DefaultPromise<>();
+
 		public JobSubmitterRunnable(
 				ActorSystem actorSystem,
 				ForkableFlinkMiniCluster cluster,
@@ -255,39 +266,23 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 		@Override
 		public void run() {
-			while (!finished) {
-				try {
-					LeaderRetrievalService lrService =
-							LeaderRetrievalUtils.createLeaderRetrievalService(
-									cluster.configuration());
-
-					ActorGateway jobManagerGateway =
-							LeaderRetrievalUtils.retrieveLeaderGateway(
-									lrService,
-									clientActorSystem,
-									timeout);
-
-					JobClient.submitJobAndWait(
-							clientActorSystem,
-							jobManagerGateway,
-							graph,
-							timeout,
-							false,
-							getClass().getClassLoader());
-
-					finished = true;
-				}
-				catch (JobExecutionException e) {
-					// This was expected, so just try again to submit the job
-				}
-				catch (LeaderRetrievalException e) {
-					// This can also happen, so just try again to submit the job
-				}
-				catch (Exception e) {
-					// This was not expected... fail the test case
-					e.printStackTrace();
-					fail("Caught unexpected exception in job submission test case.");
-				}
+			try {
+				LeaderRetrievalService lrService =
+						LeaderRetrievalUtils.createLeaderRetrievalService(
+								cluster.configuration());
+
+				JobExecutionResult result = JobClient.submitJobAndWait(
+						clientActorSystem,
+						lrService,
+						graph,
+						timeout,
+						false,
+						getClass().getClassLoader());
+
+				resultPromise.success(result);
+			} catch (Exception e) {
+				// This was not expected... fail the test case
+				resultPromise.failure(e);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/d18f5809/flink-tests/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/log4j-test.properties b/flink-tests/src/test/resources/log4j-test.properties
index 9d29841..3ba6d1d 100644
--- a/flink-tests/src/test/resources/log4j-test.properties
+++ b/flink-tests/src/test/resources/log4j-test.properties
@@ -18,7 +18,8 @@
 
 # Set root logger level to OFF to not flood build logs
 # set manually to INFO for debugging purposes
-log4j.rootLogger=OFF, testlogger
+log4j.rootLogger=INFO, testlogger
+log4j.logger.org.apache.flink.runtime.client.JobClientActor=DEBUG
 
 # A1 is set to be a ConsoleAppender.
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
@@ -27,5 +28,5 @@ log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
 log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-log4j.logger.org.apache.zookeeper=OFF, testlogger
\ No newline at end of file
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR
+log4j.logger.org.apache.zookeeper=OFF
\ No newline at end of file


[38/47] flink git commit: [FLINK-2354] [runtime] Replace old StateHandleProvider by StateStorageHelper in ZooKeeperStateHandleStore

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/css/vendor.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/css/vendor.css b/flink-runtime-web/web-dashboard/web/css/vendor.css
new file mode 100644
index 0000000..672e07f
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/web/css/vendor.css
@@ -0,0 +1,9183 @@
+/*!
+ *  Font Awesome 4.3.0 by @davegandy - http://fontawesome.io - @fontawesome
+ *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
+ */
+/* FONT PATH
+ * -------------------------- */
+@font-face {
+  font-family: 'FontAwesome';
+  src: url('../fonts/fontawesome-webfont.eot?v=4.3.0');
+  src: url('../fonts/fontawesome-webfont.eot?#iefix&v=4.3.0') format('embedded-opentype'), url('../fonts/fontawesome-webfont.woff2?v=4.3.0') format('woff2'), url('../fonts/fontawesome-webfont.woff?v=4.3.0') format('woff'), url('../fonts/fontawesome-webfont.ttf?v=4.3.0') format('truetype'), url('../fonts/fontawesome-webfont.svg?v=4.3.0#fontawesomeregular') format('svg');
+  font-weight: normal;
+  font-style: normal;
+}
+.fa {
+  display: inline-block;
+  font: normal normal normal 14px/1 FontAwesome;
+  font-size: inherit;
+  text-rendering: auto;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+  transform: translate(0, 0);
+}
+/* makes the font 33% larger relative to the icon container */
+.fa-lg {
+  font-size: 1.33333333em;
+  line-height: 0.75em;
+  vertical-align: -15%;
+}
+.fa-2x {
+  font-size: 2em;
+}
+.fa-3x {
+  font-size: 3em;
+}
+.fa-4x {
+  font-size: 4em;
+}
+.fa-5x {
+  font-size: 5em;
+}
+.fa-fw {
+  width: 1.28571429em;
+  text-align: center;
+}
+.fa-ul {
+  padding-left: 0;
+  margin-left: 2.14285714em;
+  list-style-type: none;
+}
+.fa-ul > li {
+  position: relative;
+}
+.fa-li {
+  position: absolute;
+  left: -2.14285714em;
+  width: 2.14285714em;
+  top: 0.14285714em;
+  text-align: center;
+}
+.fa-li.fa-lg {
+  left: -1.85714286em;
+}
+.fa-border {
+  padding: .2em .25em .15em;
+  border: solid 0.08em #eeeeee;
+  border-radius: .1em;
+}
+.pull-right {
+  float: right;
+}
+.pull-left {
+  float: left;
+}
+.fa.pull-left {
+  margin-right: .3em;
+}
+.fa.pull-right {
+  margin-left: .3em;
+}
+.fa-spin {
+  -webkit-animation: fa-spin 2s infinite linear;
+  animation: fa-spin 2s infinite linear;
+}
+.fa-pulse {
+  -webkit-animation: fa-spin 1s infinite steps(8);
+  animation: fa-spin 1s infinite steps(8);
+}
+@-webkit-keyframes fa-spin {
+  0% {
+    -webkit-transform: rotate(0deg);
+    transform: rotate(0deg);
+  }
+  100% {
+    -webkit-transform: rotate(359deg);
+    transform: rotate(359deg);
+  }
+}
+@keyframes fa-spin {
+  0% {
+    -webkit-transform: rotate(0deg);
+    transform: rotate(0deg);
+  }
+  100% {
+    -webkit-transform: rotate(359deg);
+    transform: rotate(359deg);
+  }
+}
+.fa-rotate-90 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=1);
+  -webkit-transform: rotate(90deg);
+  -ms-transform: rotate(90deg);
+  transform: rotate(90deg);
+}
+.fa-rotate-180 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2);
+  -webkit-transform: rotate(180deg);
+  -ms-transform: rotate(180deg);
+  transform: rotate(180deg);
+}
+.fa-rotate-270 {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=3);
+  -webkit-transform: rotate(270deg);
+  -ms-transform: rotate(270deg);
+  transform: rotate(270deg);
+}
+.fa-flip-horizontal {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);
+  -webkit-transform: scale(-1, 1);
+  -ms-transform: scale(-1, 1);
+  transform: scale(-1, 1);
+}
+.fa-flip-vertical {
+  filter: progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);
+  -webkit-transform: scale(1, -1);
+  -ms-transform: scale(1, -1);
+  transform: scale(1, -1);
+}
+:root .fa-rotate-90,
+:root .fa-rotate-180,
+:root .fa-rotate-270,
+:root .fa-flip-horizontal,
+:root .fa-flip-vertical {
+  filter: none;
+}
+.fa-stack {
+  position: relative;
+  display: inline-block;
+  width: 2em;
+  height: 2em;
+  line-height: 2em;
+  vertical-align: middle;
+}
+.fa-stack-1x,
+.fa-stack-2x {
+  position: absolute;
+  left: 0;
+  width: 100%;
+  text-align: center;
+}
+.fa-stack-1x {
+  line-height: inherit;
+}
+.fa-stack-2x {
+  font-size: 2em;
+}
+.fa-inverse {
+  color: #ffffff;
+}
+/* Font Awesome uses the Unicode Private Use Area (PUA) to ensure screen
+   readers do not read off random characters that represent icons */
+.fa-glass:before {
+  content: "\f000";
+}
+.fa-music:before {
+  content: "\f001";
+}
+.fa-search:before {
+  content: "\f002";
+}
+.fa-envelope-o:before {
+  content: "\f003";
+}
+.fa-heart:before {
+  content: "\f004";
+}
+.fa-star:before {
+  content: "\f005";
+}
+.fa-star-o:before {
+  content: "\f006";
+}
+.fa-user:before {
+  content: "\f007";
+}
+.fa-film:before {
+  content: "\f008";
+}
+.fa-th-large:before {
+  content: "\f009";
+}
+.fa-th:before {
+  content: "\f00a";
+}
+.fa-th-list:before {
+  content: "\f00b";
+}
+.fa-check:before {
+  content: "\f00c";
+}
+.fa-remove:before,
+.fa-close:before,
+.fa-times:before {
+  content: "\f00d";
+}
+.fa-search-plus:before {
+  content: "\f00e";
+}
+.fa-search-minus:before {
+  content: "\f010";
+}
+.fa-power-off:before {
+  content: "\f011";
+}
+.fa-signal:before {
+  content: "\f012";
+}
+.fa-gear:before,
+.fa-cog:before {
+  content: "\f013";
+}
+.fa-trash-o:before {
+  content: "\f014";
+}
+.fa-home:before {
+  content: "\f015";
+}
+.fa-file-o:before {
+  content: "\f016";
+}
+.fa-clock-o:before {
+  content: "\f017";
+}
+.fa-road:before {
+  content: "\f018";
+}
+.fa-download:before {
+  content: "\f019";
+}
+.fa-arrow-circle-o-down:before {
+  content: "\f01a";
+}
+.fa-arrow-circle-o-up:before {
+  content: "\f01b";
+}
+.fa-inbox:before {
+  content: "\f01c";
+}
+.fa-play-circle-o:before {
+  content: "\f01d";
+}
+.fa-rotate-right:before,
+.fa-repeat:before {
+  content: "\f01e";
+}
+.fa-refresh:before {
+  content: "\f021";
+}
+.fa-list-alt:before {
+  content: "\f022";
+}
+.fa-lock:before {
+  content: "\f023";
+}
+.fa-flag:before {
+  content: "\f024";
+}
+.fa-headphones:before {
+  content: "\f025";
+}
+.fa-volume-off:before {
+  content: "\f026";
+}
+.fa-volume-down:before {
+  content: "\f027";
+}
+.fa-volume-up:before {
+  content: "\f028";
+}
+.fa-qrcode:before {
+  content: "\f029";
+}
+.fa-barcode:before {
+  content: "\f02a";
+}
+.fa-tag:before {
+  content: "\f02b";
+}
+.fa-tags:before {
+  content: "\f02c";
+}
+.fa-book:before {
+  content: "\f02d";
+}
+.fa-bookmark:before {
+  content: "\f02e";
+}
+.fa-print:before {
+  content: "\f02f";
+}
+.fa-camera:before {
+  content: "\f030";
+}
+.fa-font:before {
+  content: "\f031";
+}
+.fa-bold:before {
+  content: "\f032";
+}
+.fa-italic:before {
+  content: "\f033";
+}
+.fa-text-height:before {
+  content: "\f034";
+}
+.fa-text-width:before {
+  content: "\f035";
+}
+.fa-align-left:before {
+  content: "\f036";
+}
+.fa-align-center:before {
+  content: "\f037";
+}
+.fa-align-right:before {
+  content: "\f038";
+}
+.fa-align-justify:before {
+  content: "\f039";
+}
+.fa-list:before {
+  content: "\f03a";
+}
+.fa-dedent:before,
+.fa-outdent:before {
+  content: "\f03b";
+}
+.fa-indent:before {
+  content: "\f03c";
+}
+.fa-video-camera:before {
+  content: "\f03d";
+}
+.fa-photo:before,
+.fa-image:before,
+.fa-picture-o:before {
+  content: "\f03e";
+}
+.fa-pencil:before {
+  content: "\f040";
+}
+.fa-map-marker:before {
+  content: "\f041";
+}
+.fa-adjust:before {
+  content: "\f042";
+}
+.fa-tint:before {
+  content: "\f043";
+}
+.fa-edit:before,
+.fa-pencil-square-o:before {
+  content: "\f044";
+}
+.fa-share-square-o:before {
+  content: "\f045";
+}
+.fa-check-square-o:before {
+  content: "\f046";
+}
+.fa-arrows:before {
+  content: "\f047";
+}
+.fa-step-backward:before {
+  content: "\f048";
+}
+.fa-fast-backward:before {
+  content: "\f049";
+}
+.fa-backward:before {
+  content: "\f04a";
+}
+.fa-play:before {
+  content: "\f04b";
+}
+.fa-pause:before {
+  content: "\f04c";
+}
+.fa-stop:before {
+  content: "\f04d";
+}
+.fa-forward:before {
+  content: "\f04e";
+}
+.fa-fast-forward:before {
+  content: "\f050";
+}
+.fa-step-forward:before {
+  content: "\f051";
+}
+.fa-eject:before {
+  content: "\f052";
+}
+.fa-chevron-left:before {
+  content: "\f053";
+}
+.fa-chevron-right:before {
+  content: "\f054";
+}
+.fa-plus-circle:before {
+  content: "\f055";
+}
+.fa-minus-circle:before {
+  content: "\f056";
+}
+.fa-times-circle:before {
+  content: "\f057";
+}
+.fa-check-circle:before {
+  content: "\f058";
+}
+.fa-question-circle:before {
+  content: "\f059";
+}
+.fa-info-circle:before {
+  content: "\f05a";
+}
+.fa-crosshairs:before {
+  content: "\f05b";
+}
+.fa-times-circle-o:before {
+  content: "\f05c";
+}
+.fa-check-circle-o:before {
+  content: "\f05d";
+}
+.fa-ban:before {
+  content: "\f05e";
+}
+.fa-arrow-left:before {
+  content: "\f060";
+}
+.fa-arrow-right:before {
+  content: "\f061";
+}
+.fa-arrow-up:before {
+  content: "\f062";
+}
+.fa-arrow-down:before {
+  content: "\f063";
+}
+.fa-mail-forward:before,
+.fa-share:before {
+  content: "\f064";
+}
+.fa-expand:before {
+  content: "\f065";
+}
+.fa-compress:before {
+  content: "\f066";
+}
+.fa-plus:before {
+  content: "\f067";
+}
+.fa-minus:before {
+  content: "\f068";
+}
+.fa-asterisk:before {
+  content: "\f069";
+}
+.fa-exclamation-circle:before {
+  content: "\f06a";
+}
+.fa-gift:before {
+  content: "\f06b";
+}
+.fa-leaf:before {
+  content: "\f06c";
+}
+.fa-fire:before {
+  content: "\f06d";
+}
+.fa-eye:before {
+  content: "\f06e";
+}
+.fa-eye-slash:before {
+  content: "\f070";
+}
+.fa-warning:before,
+.fa-exclamation-triangle:before {
+  content: "\f071";
+}
+.fa-plane:before {
+  content: "\f072";
+}
+.fa-calendar:before {
+  content: "\f073";
+}
+.fa-random:before {
+  content: "\f074";
+}
+.fa-comment:before {
+  content: "\f075";
+}
+.fa-magnet:before {
+  content: "\f076";
+}
+.fa-chevron-up:before {
+  content: "\f077";
+}
+.fa-chevron-down:before {
+  content: "\f078";
+}
+.fa-retweet:before {
+  content: "\f079";
+}
+.fa-shopping-cart:before {
+  content: "\f07a";
+}
+.fa-folder:before {
+  content: "\f07b";
+}
+.fa-folder-open:before {
+  content: "\f07c";
+}
+.fa-arrows-v:before {
+  content: "\f07d";
+}
+.fa-arrows-h:before {
+  content: "\f07e";
+}
+.fa-bar-chart-o:before,
+.fa-bar-chart:before {
+  content: "\f080";
+}
+.fa-twitter-square:before {
+  content: "\f081";
+}
+.fa-facebook-square:before {
+  content: "\f082";
+}
+.fa-camera-retro:before {
+  content: "\f083";
+}
+.fa-key:before {
+  content: "\f084";
+}
+.fa-gears:before,
+.fa-cogs:before {
+  content: "\f085";
+}
+.fa-comments:before {
+  content: "\f086";
+}
+.fa-thumbs-o-up:before {
+  content: "\f087";
+}
+.fa-thumbs-o-down:before {
+  content: "\f088";
+}
+.fa-star-half:before {
+  content: "\f089";
+}
+.fa-heart-o:before {
+  content: "\f08a";
+}
+.fa-sign-out:before {
+  content: "\f08b";
+}
+.fa-linkedin-square:before {
+  content: "\f08c";
+}
+.fa-thumb-tack:before {
+  content: "\f08d";
+}
+.fa-external-link:before {
+  content: "\f08e";
+}
+.fa-sign-in:before {
+  content: "\f090";
+}
+.fa-trophy:before {
+  content: "\f091";
+}
+.fa-github-square:before {
+  content: "\f092";
+}
+.fa-upload:before {
+  content: "\f093";
+}
+.fa-lemon-o:before {
+  content: "\f094";
+}
+.fa-phone:before {
+  content: "\f095";
+}
+.fa-square-o:before {
+  content: "\f096";
+}
+.fa-bookmark-o:before {
+  content: "\f097";
+}
+.fa-phone-square:before {
+  content: "\f098";
+}
+.fa-twitter:before {
+  content: "\f099";
+}
+.fa-facebook-f:before,
+.fa-facebook:before {
+  content: "\f09a";
+}
+.fa-github:before {
+  content: "\f09b";
+}
+.fa-unlock:before {
+  content: "\f09c";
+}
+.fa-credit-card:before {
+  content: "\f09d";
+}
+.fa-rss:before {
+  content: "\f09e";
+}
+.fa-hdd-o:before {
+  content: "\f0a0";
+}
+.fa-bullhorn:before {
+  content: "\f0a1";
+}
+.fa-bell:before {
+  content: "\f0f3";
+}
+.fa-certificate:before {
+  content: "\f0a3";
+}
+.fa-hand-o-right:before {
+  content: "\f0a4";
+}
+.fa-hand-o-left:before {
+  content: "\f0a5";
+}
+.fa-hand-o-up:before {
+  content: "\f0a6";
+}
+.fa-hand-o-down:before {
+  content: "\f0a7";
+}
+.fa-arrow-circle-left:before {
+  content: "\f0a8";
+}
+.fa-arrow-circle-right:before {
+  content: "\f0a9";
+}
+.fa-arrow-circle-up:before {
+  content: "\f0aa";
+}
+.fa-arrow-circle-down:before {
+  content: "\f0ab";
+}
+.fa-globe:before {
+  content: "\f0ac";
+}
+.fa-wrench:before {
+  content: "\f0ad";
+}
+.fa-tasks:before {
+  content: "\f0ae";
+}
+.fa-filter:before {
+  content: "\f0b0";
+}
+.fa-briefcase:before {
+  content: "\f0b1";
+}
+.fa-arrows-alt:before {
+  content: "\f0b2";
+}
+.fa-group:before,
+.fa-users:before {
+  content: "\f0c0";
+}
+.fa-chain:before,
+.fa-link:before {
+  content: "\f0c1";
+}
+.fa-cloud:before {
+  content: "\f0c2";
+}
+.fa-flask:before {
+  content: "\f0c3";
+}
+.fa-cut:before,
+.fa-scissors:before {
+  content: "\f0c4";
+}
+.fa-copy:before,
+.fa-files-o:before {
+  content: "\f0c5";
+}
+.fa-paperclip:before {
+  content: "\f0c6";
+}
+.fa-save:before,
+.fa-floppy-o:before {
+  content: "\f0c7";
+}
+.fa-square:before {
+  content: "\f0c8";
+}
+.fa-navicon:before,
+.fa-reorder:before,
+.fa-bars:before {
+  content: "\f0c9";
+}
+.fa-list-ul:before {
+  content: "\f0ca";
+}
+.fa-list-ol:before {
+  content: "\f0cb";
+}
+.fa-strikethrough:before {
+  content: "\f0cc";
+}
+.fa-underline:before {
+  content: "\f0cd";
+}
+.fa-table:before {
+  content: "\f0ce";
+}
+.fa-magic:before {
+  content: "\f0d0";
+}
+.fa-truck:before {
+  content: "\f0d1";
+}
+.fa-pinterest:before {
+  content: "\f0d2";
+}
+.fa-pinterest-square:before {
+  content: "\f0d3";
+}
+.fa-google-plus-square:before {
+  content: "\f0d4";
+}
+.fa-google-plus:before {
+  content: "\f0d5";
+}
+.fa-money:before {
+  content: "\f0d6";
+}
+.fa-caret-down:before {
+  content: "\f0d7";
+}
+.fa-caret-up:before {
+  content: "\f0d8";
+}
+.fa-caret-left:before {
+  content: "\f0d9";
+}
+.fa-caret-right:before {
+  content: "\f0da";
+}
+.fa-columns:before {
+  content: "\f0db";
+}
+.fa-unsorted:before,
+.fa-sort:before {
+  content: "\f0dc";
+}
+.fa-sort-down:before,
+.fa-sort-desc:before {
+  content: "\f0dd";
+}
+.fa-sort-up:before,
+.fa-sort-asc:before {
+  content: "\f0de";
+}
+.fa-envelope:before {
+  content: "\f0e0";
+}
+.fa-linkedin:before {
+  content: "\f0e1";
+}
+.fa-rotate-left:before,
+.fa-undo:before {
+  content: "\f0e2";
+}
+.fa-legal:before,
+.fa-gavel:before {
+  content: "\f0e3";
+}
+.fa-dashboard:before,
+.fa-tachometer:before {
+  content: "\f0e4";
+}
+.fa-comment-o:before {
+  content: "\f0e5";
+}
+.fa-comments-o:before {
+  content: "\f0e6";
+}
+.fa-flash:before,
+.fa-bolt:before {
+  content: "\f0e7";
+}
+.fa-sitemap:before {
+  content: "\f0e8";
+}
+.fa-umbrella:before {
+  content: "\f0e9";
+}
+.fa-paste:before,
+.fa-clipboard:before {
+  content: "\f0ea";
+}
+.fa-lightbulb-o:before {
+  content: "\f0eb";
+}
+.fa-exchange:before {
+  content: "\f0ec";
+}
+.fa-cloud-download:before {
+  content: "\f0ed";
+}
+.fa-cloud-upload:before {
+  content: "\f0ee";
+}
+.fa-user-md:before {
+  content: "\f0f0";
+}
+.fa-stethoscope:before {
+  content: "\f0f1";
+}
+.fa-suitcase:before {
+  content: "\f0f2";
+}
+.fa-bell-o:before {
+  content: "\f0a2";
+}
+.fa-coffee:before {
+  content: "\f0f4";
+}
+.fa-cutlery:before {
+  content: "\f0f5";
+}
+.fa-file-text-o:before {
+  content: "\f0f6";
+}
+.fa-building-o:before {
+  content: "\f0f7";
+}
+.fa-hospital-o:before {
+  content: "\f0f8";
+}
+.fa-ambulance:before {
+  content: "\f0f9";
+}
+.fa-medkit:before {
+  content: "\f0fa";
+}
+.fa-fighter-jet:before {
+  content: "\f0fb";
+}
+.fa-beer:before {
+  content: "\f0fc";
+}
+.fa-h-square:before {
+  content: "\f0fd";
+}
+.fa-plus-square:before {
+  content: "\f0fe";
+}
+.fa-angle-double-left:before {
+  content: "\f100";
+}
+.fa-angle-double-right:before {
+  content: "\f101";
+}
+.fa-angle-double-up:before {
+  content: "\f102";
+}
+.fa-angle-double-down:before {
+  content: "\f103";
+}
+.fa-angle-left:before {
+  content: "\f104";
+}
+.fa-angle-right:before {
+  content: "\f105";
+}
+.fa-angle-up:before {
+  content: "\f106";
+}
+.fa-angle-down:before {
+  content: "\f107";
+}
+.fa-desktop:before {
+  content: "\f108";
+}
+.fa-laptop:before {
+  content: "\f109";
+}
+.fa-tablet:before {
+  content: "\f10a";
+}
+.fa-mobile-phone:before,
+.fa-mobile:before {
+  content: "\f10b";
+}
+.fa-circle-o:before {
+  content: "\f10c";
+}
+.fa-quote-left:before {
+  content: "\f10d";
+}
+.fa-quote-right:before {
+  content: "\f10e";
+}
+.fa-spinner:before {
+  content: "\f110";
+}
+.fa-circle:before {
+  content: "\f111";
+}
+.fa-mail-reply:before,
+.fa-reply:before {
+  content: "\f112";
+}
+.fa-github-alt:before {
+  content: "\f113";
+}
+.fa-folder-o:before {
+  content: "\f114";
+}
+.fa-folder-open-o:before {
+  content: "\f115";
+}
+.fa-smile-o:before {
+  content: "\f118";
+}
+.fa-frown-o:before {
+  content: "\f119";
+}
+.fa-meh-o:before {
+  content: "\f11a";
+}
+.fa-gamepad:before {
+  content: "\f11b";
+}
+.fa-keyboard-o:before {
+  content: "\f11c";
+}
+.fa-flag-o:before {
+  content: "\f11d";
+}
+.fa-flag-checkered:before {
+  content: "\f11e";
+}
+.fa-terminal:before {
+  content: "\f120";
+}
+.fa-code:before {
+  content: "\f121";
+}
+.fa-mail-reply-all:before,
+.fa-reply-all:before {
+  content: "\f122";
+}
+.fa-star-half-empty:before,
+.fa-star-half-full:before,
+.fa-star-half-o:before {
+  content: "\f123";
+}
+.fa-location-arrow:before {
+  content: "\f124";
+}
+.fa-crop:before {
+  content: "\f125";
+}
+.fa-code-fork:before {
+  content: "\f126";
+}
+.fa-unlink:before,
+.fa-chain-broken:before {
+  content: "\f127";
+}
+.fa-question:before {
+  content: "\f128";
+}
+.fa-info:before {
+  content: "\f129";
+}
+.fa-exclamation:before {
+  content: "\f12a";
+}
+.fa-superscript:before {
+  content: "\f12b";
+}
+.fa-subscript:before {
+  content: "\f12c";
+}
+.fa-eraser:before {
+  content: "\f12d";
+}
+.fa-puzzle-piece:before {
+  content: "\f12e";
+}
+.fa-microphone:before {
+  content: "\f130";
+}
+.fa-microphone-slash:before {
+  content: "\f131";
+}
+.fa-shield:before {
+  content: "\f132";
+}
+.fa-calendar-o:before {
+  content: "\f133";
+}
+.fa-fire-extinguisher:before {
+  content: "\f134";
+}
+.fa-rocket:before {
+  content: "\f135";
+}
+.fa-maxcdn:before {
+  content: "\f136";
+}
+.fa-chevron-circle-left:before {
+  content: "\f137";
+}
+.fa-chevron-circle-right:before {
+  content: "\f138";
+}
+.fa-chevron-circle-up:before {
+  content: "\f139";
+}
+.fa-chevron-circle-down:before {
+  content: "\f13a";
+}
+.fa-html5:before {
+  content: "\f13b";
+}
+.fa-css3:before {
+  content: "\f13c";
+}
+.fa-anchor:before {
+  content: "\f13d";
+}
+.fa-unlock-alt:before {
+  content: "\f13e";
+}
+.fa-bullseye:before {
+  content: "\f140";
+}
+.fa-ellipsis-h:before {
+  content: "\f141";
+}
+.fa-ellipsis-v:before {
+  content: "\f142";
+}
+.fa-rss-square:before {
+  content: "\f143";
+}
+.fa-play-circle:before {
+  content: "\f144";
+}
+.fa-ticket:before {
+  content: "\f145";
+}
+.fa-minus-square:before {
+  content: "\f146";
+}
+.fa-minus-square-o:before {
+  content: "\f147";
+}
+.fa-level-up:before {
+  content: "\f148";
+}
+.fa-level-down:before {
+  content: "\f149";
+}
+.fa-check-square:before {
+  content: "\f14a";
+}
+.fa-pencil-square:before {
+  content: "\f14b";
+}
+.fa-external-link-square:before {
+  content: "\f14c";
+}
+.fa-share-square:before {
+  content: "\f14d";
+}
+.fa-compass:before {
+  content: "\f14e";
+}
+.fa-toggle-down:before,
+.fa-caret-square-o-down:before {
+  content: "\f150";
+}
+.fa-toggle-up:before,
+.fa-caret-square-o-up:before {
+  content: "\f151";
+}
+.fa-toggle-right:before,
+.fa-caret-square-o-right:before {
+  content: "\f152";
+}
+.fa-euro:before,
+.fa-eur:before {
+  content: "\f153";
+}
+.fa-gbp:before {
+  content: "\f154";
+}
+.fa-dollar:before,
+.fa-usd:before {
+  content: "\f155";
+}
+.fa-rupee:before,
+.fa-inr:before {
+  content: "\f156";
+}
+.fa-cny:before,
+.fa-rmb:before,
+.fa-yen:before,
+.fa-jpy:before {
+  content: "\f157";
+}
+.fa-ruble:before,
+.fa-rouble:before,
+.fa-rub:before {
+  content: "\f158";
+}
+.fa-won:before,
+.fa-krw:before {
+  content: "\f159";
+}
+.fa-bitcoin:before,
+.fa-btc:before {
+  content: "\f15a";
+}
+.fa-file:before {
+  content: "\f15b";
+}
+.fa-file-text:before {
+  content: "\f15c";
+}
+.fa-sort-alpha-asc:before {
+  content: "\f15d";
+}
+.fa-sort-alpha-desc:before {
+  content: "\f15e";
+}
+.fa-sort-amount-asc:before {
+  content: "\f160";
+}
+.fa-sort-amount-desc:before {
+  content: "\f161";
+}
+.fa-sort-numeric-asc:before {
+  content: "\f162";
+}
+.fa-sort-numeric-desc:before {
+  content: "\f163";
+}
+.fa-thumbs-up:before {
+  content: "\f164";
+}
+.fa-thumbs-down:before {
+  content: "\f165";
+}
+.fa-youtube-square:before {
+  content: "\f166";
+}
+.fa-youtube:before {
+  content: "\f167";
+}
+.fa-xing:before {
+  content: "\f168";
+}
+.fa-xing-square:before {
+  content: "\f169";
+}
+.fa-youtube-play:before {
+  content: "\f16a";
+}
+.fa-dropbox:before {
+  content: "\f16b";
+}
+.fa-stack-overflow:before {
+  content: "\f16c";
+}
+.fa-instagram:before {
+  content: "\f16d";
+}
+.fa-flickr:before {
+  content: "\f16e";
+}
+.fa-adn:before {
+  content: "\f170";
+}
+.fa-bitbucket:before {
+  content: "\f171";
+}
+.fa-bitbucket-square:before {
+  content: "\f172";
+}
+.fa-tumblr:before {
+  content: "\f173";
+}
+.fa-tumblr-square:before {
+  content: "\f174";
+}
+.fa-long-arrow-down:before {
+  content: "\f175";
+}
+.fa-long-arrow-up:before {
+  content: "\f176";
+}
+.fa-long-arrow-left:before {
+  content: "\f177";
+}
+.fa-long-arrow-right:before {
+  content: "\f178";
+}
+.fa-apple:before {
+  content: "\f179";
+}
+.fa-windows:before {
+  content: "\f17a";
+}
+.fa-android:before {
+  content: "\f17b";
+}
+.fa-linux:before {
+  content: "\f17c";
+}
+.fa-dribbble:before {
+  content: "\f17d";
+}
+.fa-skype:before {
+  content: "\f17e";
+}
+.fa-foursquare:before {
+  content: "\f180";
+}
+.fa-trello:before {
+  content: "\f181";
+}
+.fa-female:before {
+  content: "\f182";
+}
+.fa-male:before {
+  content: "\f183";
+}
+.fa-gittip:before,
+.fa-gratipay:before {
+  content: "\f184";
+}
+.fa-sun-o:before {
+  content: "\f185";
+}
+.fa-moon-o:before {
+  content: "\f186";
+}
+.fa-archive:before {
+  content: "\f187";
+}
+.fa-bug:before {
+  content: "\f188";
+}
+.fa-vk:before {
+  content: "\f189";
+}
+.fa-weibo:before {
+  content: "\f18a";
+}
+.fa-renren:before {
+  content: "\f18b";
+}
+.fa-pagelines:before {
+  content: "\f18c";
+}
+.fa-stack-exchange:before {
+  content: "\f18d";
+}
+.fa-arrow-circle-o-right:before {
+  content: "\f18e";
+}
+.fa-arrow-circle-o-left:before {
+  content: "\f190";
+}
+.fa-toggle-left:before,
+.fa-caret-square-o-left:before {
+  content: "\f191";
+}
+.fa-dot-circle-o:before {
+  content: "\f192";
+}
+.fa-wheelchair:before {
+  content: "\f193";
+}
+.fa-vimeo-square:before {
+  content: "\f194";
+}
+.fa-turkish-lira:before,
+.fa-try:before {
+  content: "\f195";
+}
+.fa-plus-square-o:before {
+  content: "\f196";
+}
+.fa-space-shuttle:before {
+  content: "\f197";
+}
+.fa-slack:before {
+  content: "\f198";
+}
+.fa-envelope-square:before {
+  content: "\f199";
+}
+.fa-wordpress:before {
+  content: "\f19a";
+}
+.fa-openid:before {
+  content: "\f19b";
+}
+.fa-institution:before,
+.fa-bank:before,
+.fa-university:before {
+  content: "\f19c";
+}
+.fa-mortar-board:before,
+.fa-graduation-cap:before {
+  content: "\f19d";
+}
+.fa-yahoo:before {
+  content: "\f19e";
+}
+.fa-google:before {
+  content: "\f1a0";
+}
+.fa-reddit:before {
+  content: "\f1a1";
+}
+.fa-reddit-square:before {
+  content: "\f1a2";
+}
+.fa-stumbleupon-circle:before {
+  content: "\f1a3";
+}
+.fa-stumbleupon:before {
+  content: "\f1a4";
+}
+.fa-delicious:before {
+  content: "\f1a5";
+}
+.fa-digg:before {
+  content: "\f1a6";
+}
+.fa-pied-piper:before {
+  content: "\f1a7";
+}
+.fa-pied-piper-alt:before {
+  content: "\f1a8";
+}
+.fa-drupal:before {
+  content: "\f1a9";
+}
+.fa-joomla:before {
+  content: "\f1aa";
+}
+.fa-language:before {
+  content: "\f1ab";
+}
+.fa-fax:before {
+  content: "\f1ac";
+}
+.fa-building:before {
+  content: "\f1ad";
+}
+.fa-child:before {
+  content: "\f1ae";
+}
+.fa-paw:before {
+  content: "\f1b0";
+}
+.fa-spoon:before {
+  content: "\f1b1";
+}
+.fa-cube:before {
+  content: "\f1b2";
+}
+.fa-cubes:before {
+  content: "\f1b3";
+}
+.fa-behance:before {
+  content: "\f1b4";
+}
+.fa-behance-square:before {
+  content: "\f1b5";
+}
+.fa-steam:before {
+  content: "\f1b6";
+}
+.fa-steam-square:before {
+  content: "\f1b7";
+}
+.fa-recycle:before {
+  content: "\f1b8";
+}
+.fa-automobile:before,
+.fa-car:before {
+  content: "\f1b9";
+}
+.fa-cab:before,
+.fa-taxi:before {
+  content: "\f1ba";
+}
+.fa-tree:before {
+  content: "\f1bb";
+}
+.fa-spotify:before {
+  content: "\f1bc";
+}
+.fa-deviantart:before {
+  content: "\f1bd";
+}
+.fa-soundcloud:before {
+  content: "\f1be";
+}
+.fa-database:before {
+  content: "\f1c0";
+}
+.fa-file-pdf-o:before {
+  content: "\f1c1";
+}
+.fa-file-word-o:before {
+  content: "\f1c2";
+}
+.fa-file-excel-o:before {
+  content: "\f1c3";
+}
+.fa-file-powerpoint-o:before {
+  content: "\f1c4";
+}
+.fa-file-photo-o:before,
+.fa-file-picture-o:before,
+.fa-file-image-o:before {
+  content: "\f1c5";
+}
+.fa-file-zip-o:before,
+.fa-file-archive-o:before {
+  content: "\f1c6";
+}
+.fa-file-sound-o:before,
+.fa-file-audio-o:before {
+  content: "\f1c7";
+}
+.fa-file-movie-o:before,
+.fa-file-video-o:before {
+  content: "\f1c8";
+}
+.fa-file-code-o:before {
+  content: "\f1c9";
+}
+.fa-vine:before {
+  content: "\f1ca";
+}
+.fa-codepen:before {
+  content: "\f1cb";
+}
+.fa-jsfiddle:before {
+  content: "\f1cc";
+}
+.fa-life-bouy:before,
+.fa-life-buoy:before,
+.fa-life-saver:before,
+.fa-support:before,
+.fa-life-ring:before {
+  content: "\f1cd";
+}
+.fa-circle-o-notch:before {
+  content: "\f1ce";
+}
+.fa-ra:before,
+.fa-rebel:before {
+  content: "\f1d0";
+}
+.fa-ge:before,
+.fa-empire:before {
+  content: "\f1d1";
+}
+.fa-git-square:before {
+  content: "\f1d2";
+}
+.fa-git:before {
+  content: "\f1d3";
+}
+.fa-hacker-news:before {
+  content: "\f1d4";
+}
+.fa-tencent-weibo:before {
+  content: "\f1d5";
+}
+.fa-qq:before {
+  content: "\f1d6";
+}
+.fa-wechat:before,
+.fa-weixin:before {
+  content: "\f1d7";
+}
+.fa-send:before,
+.fa-paper-plane:before {
+  content: "\f1d8";
+}
+.fa-send-o:before,
+.fa-paper-plane-o:before {
+  content: "\f1d9";
+}
+.fa-history:before {
+  content: "\f1da";
+}
+.fa-genderless:before,
+.fa-circle-thin:before {
+  content: "\f1db";
+}
+.fa-header:before {
+  content: "\f1dc";
+}
+.fa-paragraph:before {
+  content: "\f1dd";
+}
+.fa-sliders:before {
+  content: "\f1de";
+}
+.fa-share-alt:before {
+  content: "\f1e0";
+}
+.fa-share-alt-square:before {
+  content: "\f1e1";
+}
+.fa-bomb:before {
+  content: "\f1e2";
+}
+.fa-soccer-ball-o:before,
+.fa-futbol-o:before {
+  content: "\f1e3";
+}
+.fa-tty:before {
+  content: "\f1e4";
+}
+.fa-binoculars:before {
+  content: "\f1e5";
+}
+.fa-plug:before {
+  content: "\f1e6";
+}
+.fa-slideshare:before {
+  content: "\f1e7";
+}
+.fa-twitch:before {
+  content: "\f1e8";
+}
+.fa-yelp:before {
+  content: "\f1e9";
+}
+.fa-newspaper-o:before {
+  content: "\f1ea";
+}
+.fa-wifi:before {
+  content: "\f1eb";
+}
+.fa-calculator:before {
+  content: "\f1ec";
+}
+.fa-paypal:before {
+  content: "\f1ed";
+}
+.fa-google-wallet:before {
+  content: "\f1ee";
+}
+.fa-cc-visa:before {
+  content: "\f1f0";
+}
+.fa-cc-mastercard:before {
+  content: "\f1f1";
+}
+.fa-cc-discover:before {
+  content: "\f1f2";
+}
+.fa-cc-amex:before {
+  content: "\f1f3";
+}
+.fa-cc-paypal:before {
+  content: "\f1f4";
+}
+.fa-cc-stripe:before {
+  content: "\f1f5";
+}
+.fa-bell-slash:before {
+  content: "\f1f6";
+}
+.fa-bell-slash-o:before {
+  content: "\f1f7";
+}
+.fa-trash:before {
+  content: "\f1f8";
+}
+.fa-copyright:before {
+  content: "\f1f9";
+}
+.fa-at:before {
+  content: "\f1fa";
+}
+.fa-eyedropper:before {
+  content: "\f1fb";
+}
+.fa-paint-brush:before {
+  content: "\f1fc";
+}
+.fa-birthday-cake:before {
+  content: "\f1fd";
+}
+.fa-area-chart:before {
+  content: "\f1fe";
+}
+.fa-pie-chart:before {
+  content: "\f200";
+}
+.fa-line-chart:before {
+  content: "\f201";
+}
+.fa-lastfm:before {
+  content: "\f202";
+}
+.fa-lastfm-square:before {
+  content: "\f203";
+}
+.fa-toggle-off:before {
+  content: "\f204";
+}
+.fa-toggle-on:before {
+  content: "\f205";
+}
+.fa-bicycle:before {
+  content: "\f206";
+}
+.fa-bus:before {
+  content: "\f207";
+}
+.fa-ioxhost:before {
+  content: "\f208";
+}
+.fa-angellist:before {
+  content: "\f209";
+}
+.fa-cc:before {
+  content: "\f20a";
+}
+.fa-shekel:before,
+.fa-sheqel:before,
+.fa-ils:before {
+  content: "\f20b";
+}
+.fa-meanpath:before {
+  content: "\f20c";
+}
+.fa-buysellads:before {
+  content: "\f20d";
+}
+.fa-connectdevelop:before {
+  content: "\f20e";
+}
+.fa-dashcube:before {
+  content: "\f210";
+}
+.fa-forumbee:before {
+  content: "\f211";
+}
+.fa-leanpub:before {
+  content: "\f212";
+}
+.fa-sellsy:before {
+  content: "\f213";
+}
+.fa-shirtsinbulk:before {
+  content: "\f214";
+}
+.fa-simplybuilt:before {
+  content: "\f215";
+}
+.fa-skyatlas:before {
+  content: "\f216";
+}
+.fa-cart-plus:before {
+  content: "\f217";
+}
+.fa-cart-arrow-down:before {
+  content: "\f218";
+}
+.fa-diamond:before {
+  content: "\f219";
+}
+.fa-ship:before {
+  content: "\f21a";
+}
+.fa-user-secret:before {
+  content: "\f21b";
+}
+.fa-motorcycle:before {
+  content: "\f21c";
+}
+.fa-street-view:before {
+  content: "\f21d";
+}
+.fa-heartbeat:before {
+  content: "\f21e";
+}
+.fa-venus:before {
+  content: "\f221";
+}
+.fa-mars:before {
+  content: "\f222";
+}
+.fa-mercury:before {
+  content: "\f223";
+}
+.fa-transgender:before {
+  content: "\f224";
+}
+.fa-transgender-alt:before {
+  content: "\f225";
+}
+.fa-venus-double:before {
+  content: "\f226";
+}
+.fa-mars-double:before {
+  content: "\f227";
+}
+.fa-venus-mars:before {
+  content: "\f228";
+}
+.fa-mars-stroke:before {
+  content: "\f229";
+}
+.fa-mars-stroke-v:before {
+  content: "\f22a";
+}
+.fa-mars-stroke-h:before {
+  content: "\f22b";
+}
+.fa-neuter:before {
+  content: "\f22c";
+}
+.fa-facebook-official:before {
+  content: "\f230";
+}
+.fa-pinterest-p:before {
+  content: "\f231";
+}
+.fa-whatsapp:before {
+  content: "\f232";
+}
+.fa-server:before {
+  content: "\f233";
+}
+.fa-user-plus:before {
+  content: "\f234";
+}
+.fa-user-times:before {
+  content: "\f235";
+}
+.fa-hotel:before,
+.fa-bed:before {
+  content: "\f236";
+}
+.fa-viacoin:before {
+  content: "\f237";
+}
+.fa-train:before {
+  content: "\f238";
+}
+.fa-subway:before {
+  content: "\f239";
+}
+.fa-medium:before {
+  content: "\f23a";
+}
+
+/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */
+html {
+  font-family: sans-serif;
+  -ms-text-size-adjust: 100%;
+  -webkit-text-size-adjust: 100%;
+}
+body {
+  margin: 0;
+}
+article,
+aside,
+details,
+figcaption,
+figure,
+footer,
+header,
+hgroup,
+main,
+menu,
+nav,
+section,
+summary {
+  display: block;
+}
+audio,
+canvas,
+progress,
+video {
+  display: inline-block;
+  vertical-align: baseline;
+}
+audio:not([controls]) {
+  display: none;
+  height: 0;
+}
+[hidden],
+template {
+  display: none;
+}
+a {
+  background-color: transparent;
+}
+a:active,
+a:hover {
+  outline: 0;
+}
+abbr[title] {
+  border-bottom: 1px dotted;
+}
+b,
+strong {
+  font-weight: bold;
+}
+dfn {
+  font-style: italic;
+}
+h1 {
+  font-size: 2em;
+  margin: 0.67em 0;
+}
+mark {
+  background: #ff0;
+  color: #000;
+}
+small {
+  font-size: 80%;
+}
+sub,
+sup {
+  font-size: 75%;
+  line-height: 0;
+  position: relative;
+  vertical-align: baseline;
+}
+sup {
+  top: -0.5em;
+}
+sub {
+  bottom: -0.25em;
+}
+img {
+  border: 0;
+}
+svg:not(:root) {
+  overflow: hidden;
+}
+figure {
+  margin: 1em 40px;
+}
+hr {
+  box-sizing: content-box;
+  height: 0;
+}
+pre {
+  overflow: auto;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: monospace, monospace;
+  font-size: 1em;
+}
+button,
+input,
+optgroup,
+select,
+textarea {
+  color: inherit;
+  font: inherit;
+  margin: 0;
+}
+button {
+  overflow: visible;
+}
+button,
+select {
+  text-transform: none;
+}
+button,
+html input[type="button"],
+input[type="reset"],
+input[type="submit"] {
+  -webkit-appearance: button;
+  cursor: pointer;
+}
+button[disabled],
+html input[disabled] {
+  cursor: default;
+}
+button::-moz-focus-inner,
+input::-moz-focus-inner {
+  border: 0;
+  padding: 0;
+}
+input {
+  line-height: normal;
+}
+input[type="checkbox"],
+input[type="radio"] {
+  box-sizing: border-box;
+  padding: 0;
+}
+input[type="number"]::-webkit-inner-spin-button,
+input[type="number"]::-webkit-outer-spin-button {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: textfield;
+  box-sizing: content-box;
+}
+input[type="search"]::-webkit-search-cancel-button,
+input[type="search"]::-webkit-search-decoration {
+  -webkit-appearance: none;
+}
+fieldset {
+  border: 1px solid #c0c0c0;
+  margin: 0 2px;
+  padding: 0.35em 0.625em 0.75em;
+}
+legend {
+  border: 0;
+  padding: 0;
+}
+textarea {
+  overflow: auto;
+}
+optgroup {
+  font-weight: bold;
+}
+table {
+  border-collapse: collapse;
+  border-spacing: 0;
+}
+td,
+th {
+  padding: 0;
+}
+/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */
+@media print {
+  *,
+  *:before,
+  *:after {
+    background: transparent !important;
+    color: #000 !important;
+    box-shadow: none !important;
+    text-shadow: none !important;
+  }
+  a,
+  a:visited {
+    text-decoration: underline;
+  }
+  a[href]:after {
+    content: " (" attr(href) ")";
+  }
+  abbr[title]:after {
+    content: " (" attr(title) ")";
+  }
+  a[href^="#"]:after,
+  a[href^="javascript:"]:after {
+    content: "";
+  }
+  pre,
+  blockquote {
+    border: 1px solid #999;
+    page-break-inside: avoid;
+  }
+  thead {
+    display: table-header-group;
+  }
+  tr,
+  img {
+    page-break-inside: avoid;
+  }
+  img {
+    max-width: 100% !important;
+  }
+  p,
+  h2,
+  h3 {
+    orphans: 3;
+    widows: 3;
+  }
+  h2,
+  h3 {
+    page-break-after: avoid;
+  }
+  .navbar {
+    display: none;
+  }
+  .btn > .caret,
+  .dropup > .btn > .caret {
+    border-top-color: #000 !important;
+  }
+  .label {
+    border: 1px solid #000;
+  }
+  .table {
+    border-collapse: collapse !important;
+  }
+  .table td,
+  .table th {
+    background-color: #fff !important;
+  }
+  .table-bordered th,
+  .table-bordered td {
+    border: 1px solid #ddd !important;
+  }
+}
+@font-face {
+  font-family: 'Glyphicons Halflings';
+  src: url('../fonts/glyphicons-halflings-regular.eot');
+  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
+}
+.glyphicon {
+  position: relative;
+  top: 1px;
+  display: inline-block;
+  font-family: 'Glyphicons Halflings';
+  font-style: normal;
+  font-weight: normal;
+  line-height: 1;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+}
+.glyphicon-asterisk:before {
+  content: "\2a";
+}
+.glyphicon-plus:before {
+  content: "\2b";
+}
+.glyphicon-euro:before,
+.glyphicon-eur:before {
+  content: "\20ac";
+}
+.glyphicon-minus:before {
+  content: "\2212";
+}
+.glyphicon-cloud:before {
+  content: "\2601";
+}
+.glyphicon-envelope:before {
+  content: "\2709";
+}
+.glyphicon-pencil:before {
+  content: "\270f";
+}
+.glyphicon-glass:before {
+  content: "\e001";
+}
+.glyphicon-music:before {
+  content: "\e002";
+}
+.glyphicon-search:before {
+  content: "\e003";
+}
+.glyphicon-heart:before {
+  content: "\e005";
+}
+.glyphicon-star:before {
+  content: "\e006";
+}
+.glyphicon-star-empty:before {
+  content: "\e007";
+}
+.glyphicon-user:before {
+  content: "\e008";
+}
+.glyphicon-film:before {
+  content: "\e009";
+}
+.glyphicon-th-large:before {
+  content: "\e010";
+}
+.glyphicon-th:before {
+  content: "\e011";
+}
+.glyphicon-th-list:before {
+  content: "\e012";
+}
+.glyphicon-ok:before {
+  content: "\e013";
+}
+.glyphicon-remove:before {
+  content: "\e014";
+}
+.glyphicon-zoom-in:before {
+  content: "\e015";
+}
+.glyphicon-zoom-out:before {
+  content: "\e016";
+}
+.glyphicon-off:before {
+  content: "\e017";
+}
+.glyphicon-signal:before {
+  content: "\e018";
+}
+.glyphicon-cog:before {
+  content: "\e019";
+}
+.glyphicon-trash:before {
+  content: "\e020";
+}
+.glyphicon-home:before {
+  content: "\e021";
+}
+.glyphicon-file:before {
+  content: "\e022";
+}
+.glyphicon-time:before {
+  content: "\e023";
+}
+.glyphicon-road:before {
+  content: "\e024";
+}
+.glyphicon-download-alt:before {
+  content: "\e025";
+}
+.glyphicon-download:before {
+  content: "\e026";
+}
+.glyphicon-upload:before {
+  content: "\e027";
+}
+.glyphicon-inbox:before {
+  content: "\e028";
+}
+.glyphicon-play-circle:before {
+  content: "\e029";
+}
+.glyphicon-repeat:before {
+  content: "\e030";
+}
+.glyphicon-refresh:before {
+  content: "\e031";
+}
+.glyphicon-list-alt:before {
+  content: "\e032";
+}
+.glyphicon-lock:before {
+  content: "\e033";
+}
+.glyphicon-flag:before {
+  content: "\e034";
+}
+.glyphicon-headphones:before {
+  content: "\e035";
+}
+.glyphicon-volume-off:before {
+  content: "\e036";
+}
+.glyphicon-volume-down:before {
+  content: "\e037";
+}
+.glyphicon-volume-up:before {
+  content: "\e038";
+}
+.glyphicon-qrcode:before {
+  content: "\e039";
+}
+.glyphicon-barcode:before {
+  content: "\e040";
+}
+.glyphicon-tag:before {
+  content: "\e041";
+}
+.glyphicon-tags:before {
+  content: "\e042";
+}
+.glyphicon-book:before {
+  content: "\e043";
+}
+.glyphicon-bookmark:before {
+  content: "\e044";
+}
+.glyphicon-print:before {
+  content: "\e045";
+}
+.glyphicon-camera:before {
+  content: "\e046";
+}
+.glyphicon-font:before {
+  content: "\e047";
+}
+.glyphicon-bold:before {
+  content: "\e048";
+}
+.glyphicon-italic:before {
+  content: "\e049";
+}
+.glyphicon-text-height:before {
+  content: "\e050";
+}
+.glyphicon-text-width:before {
+  content: "\e051";
+}
+.glyphicon-align-left:before {
+  content: "\e052";
+}
+.glyphicon-align-center:before {
+  content: "\e053";
+}
+.glyphicon-align-right:before {
+  content: "\e054";
+}
+.glyphicon-align-justify:before {
+  content: "\e055";
+}
+.glyphicon-list:before {
+  content: "\e056";
+}
+.glyphicon-indent-left:before {
+  content: "\e057";
+}
+.glyphicon-indent-right:before {
+  content: "\e058";
+}
+.glyphicon-facetime-video:before {
+  content: "\e059";
+}
+.glyphicon-picture:before {
+  content: "\e060";
+}
+.glyphicon-map-marker:before {
+  content: "\e062";
+}
+.glyphicon-adjust:before {
+  content: "\e063";
+}
+.glyphicon-tint:before {
+  content: "\e064";
+}
+.glyphicon-edit:before {
+  content: "\e065";
+}
+.glyphicon-share:before {
+  content: "\e066";
+}
+.glyphicon-check:before {
+  content: "\e067";
+}
+.glyphicon-move:before {
+  content: "\e068";
+}
+.glyphicon-step-backward:before {
+  content: "\e069";
+}
+.glyphicon-fast-backward:before {
+  content: "\e070";
+}
+.glyphicon-backward:before {
+  content: "\e071";
+}
+.glyphicon-play:before {
+  content: "\e072";
+}
+.glyphicon-pause:before {
+  content: "\e073";
+}
+.glyphicon-stop:before {
+  content: "\e074";
+}
+.glyphicon-forward:before {
+  content: "\e075";
+}
+.glyphicon-fast-forward:before {
+  content: "\e076";
+}
+.glyphicon-step-forward:before {
+  content: "\e077";
+}
+.glyphicon-eject:before {
+  content: "\e078";
+}
+.glyphicon-chevron-left:before {
+  content: "\e079";
+}
+.glyphicon-chevron-right:before {
+  content: "\e080";
+}
+.glyphicon-plus-sign:before {
+  content: "\e081";
+}
+.glyphicon-minus-sign:before {
+  content: "\e082";
+}
+.glyphicon-remove-sign:before {
+  content: "\e083";
+}
+.glyphicon-ok-sign:before {
+  content: "\e084";
+}
+.glyphicon-question-sign:before {
+  content: "\e085";
+}
+.glyphicon-info-sign:before {
+  content: "\e086";
+}
+.glyphicon-screenshot:before {
+  content: "\e087";
+}
+.glyphicon-remove-circle:before {
+  content: "\e088";
+}
+.glyphicon-ok-circle:before {
+  content: "\e089";
+}
+.glyphicon-ban-circle:before {
+  content: "\e090";
+}
+.glyphicon-arrow-left:before {
+  content: "\e091";
+}
+.glyphicon-arrow-right:before {
+  content: "\e092";
+}
+.glyphicon-arrow-up:before {
+  content: "\e093";
+}
+.glyphicon-arrow-down:before {
+  content: "\e094";
+}
+.glyphicon-share-alt:before {
+  content: "\e095";
+}
+.glyphicon-resize-full:before {
+  content: "\e096";
+}
+.glyphicon-resize-small:before {
+  content: "\e097";
+}
+.glyphicon-exclamation-sign:before {
+  content: "\e101";
+}
+.glyphicon-gift:before {
+  content: "\e102";
+}
+.glyphicon-leaf:before {
+  content: "\e103";
+}
+.glyphicon-fire:before {
+  content: "\e104";
+}
+.glyphicon-eye-open:before {
+  content: "\e105";
+}
+.glyphicon-eye-close:before {
+  content: "\e106";
+}
+.glyphicon-warning-sign:before {
+  content: "\e107";
+}
+.glyphicon-plane:before {
+  content: "\e108";
+}
+.glyphicon-calendar:before {
+  content: "\e109";
+}
+.glyphicon-random:before {
+  content: "\e110";
+}
+.glyphicon-comment:before {
+  content: "\e111";
+}
+.glyphicon-magnet:before {
+  content: "\e112";
+}
+.glyphicon-chevron-up:before {
+  content: "\e113";
+}
+.glyphicon-chevron-down:before {
+  content: "\e114";
+}
+.glyphicon-retweet:before {
+  content: "\e115";
+}
+.glyphicon-shopping-cart:before {
+  content: "\e116";
+}
+.glyphicon-folder-close:before {
+  content: "\e117";
+}
+.glyphicon-folder-open:before {
+  content: "\e118";
+}
+.glyphicon-resize-vertical:before {
+  content: "\e119";
+}
+.glyphicon-resize-horizontal:before {
+  content: "\e120";
+}
+.glyphicon-hdd:before {
+  content: "\e121";
+}
+.glyphicon-bullhorn:before {
+  content: "\e122";
+}
+.glyphicon-bell:before {
+  content: "\e123";
+}
+.glyphicon-certificate:before {
+  content: "\e124";
+}
+.glyphicon-thumbs-up:before {
+  content: "\e125";
+}
+.glyphicon-thumbs-down:before {
+  content: "\e126";
+}
+.glyphicon-hand-right:before {
+  content: "\e127";
+}
+.glyphicon-hand-left:before {
+  content: "\e128";
+}
+.glyphicon-hand-up:before {
+  content: "\e129";
+}
+.glyphicon-hand-down:before {
+  content: "\e130";
+}
+.glyphicon-circle-arrow-right:before {
+  content: "\e131";
+}
+.glyphicon-circle-arrow-left:before {
+  content: "\e132";
+}
+.glyphicon-circle-arrow-up:before {
+  content: "\e133";
+}
+.glyphicon-circle-arrow-down:before {
+  content: "\e134";
+}
+.glyphicon-globe:before {
+  content: "\e135";
+}
+.glyphicon-wrench:before {
+  content: "\e136";
+}
+.glyphicon-tasks:before {
+  content: "\e137";
+}
+.glyphicon-filter:before {
+  content: "\e138";
+}
+.glyphicon-briefcase:before {
+  content: "\e139";
+}
+.glyphicon-fullscreen:before {
+  content: "\e140";
+}
+.glyphicon-dashboard:before {
+  content: "\e141";
+}
+.glyphicon-paperclip:before {
+  content: "\e142";
+}
+.glyphicon-heart-empty:before {
+  content: "\e143";
+}
+.glyphicon-link:before {
+  content: "\e144";
+}
+.glyphicon-phone:before {
+  content: "\e145";
+}
+.glyphicon-pushpin:before {
+  content: "\e146";
+}
+.glyphicon-usd:before {
+  content: "\e148";
+}
+.glyphicon-gbp:before {
+  content: "\e149";
+}
+.glyphicon-sort:before {
+  content: "\e150";
+}
+.glyphicon-sort-by-alphabet:before {
+  content: "\e151";
+}
+.glyphicon-sort-by-alphabet-alt:before {
+  content: "\e152";
+}
+.glyphicon-sort-by-order:before {
+  content: "\e153";
+}
+.glyphicon-sort-by-order-alt:before {
+  content: "\e154";
+}
+.glyphicon-sort-by-attributes:before {
+  content: "\e155";
+}
+.glyphicon-sort-by-attributes-alt:before {
+  content: "\e156";
+}
+.glyphicon-unchecked:before {
+  content: "\e157";
+}
+.glyphicon-expand:before {
+  content: "\e158";
+}
+.glyphicon-collapse-down:before {
+  content: "\e159";
+}
+.glyphicon-collapse-up:before {
+  content: "\e160";
+}
+.glyphicon-log-in:before {
+  content: "\e161";
+}
+.glyphicon-flash:before {
+  content: "\e162";
+}
+.glyphicon-log-out:before {
+  content: "\e163";
+}
+.glyphicon-new-window:before {
+  content: "\e164";
+}
+.glyphicon-record:before {
+  content: "\e165";
+}
+.glyphicon-save:before {
+  content: "\e166";
+}
+.glyphicon-open:before {
+  content: "\e167";
+}
+.glyphicon-saved:before {
+  content: "\e168";
+}
+.glyphicon-import:before {
+  content: "\e169";
+}
+.glyphicon-export:before {
+  content: "\e170";
+}
+.glyphicon-send:before {
+  content: "\e171";
+}
+.glyphicon-floppy-disk:before {
+  content: "\e172";
+}
+.glyphicon-floppy-saved:before {
+  content: "\e173";
+}
+.glyphicon-floppy-remove:before {
+  content: "\e174";
+}
+.glyphicon-floppy-save:before {
+  content: "\e175";
+}
+.glyphicon-floppy-open:before {
+  content: "\e176";
+}
+.glyphicon-credit-card:before {
+  content: "\e177";
+}
+.glyphicon-transfer:before {
+  content: "\e178";
+}
+.glyphicon-cutlery:before {
+  content: "\e179";
+}
+.glyphicon-header:before {
+  content: "\e180";
+}
+.glyphicon-compressed:before {
+  content: "\e181";
+}
+.glyphicon-earphone:before {
+  content: "\e182";
+}
+.glyphicon-phone-alt:before {
+  content: "\e183";
+}
+.glyphicon-tower:before {
+  content: "\e184";
+}
+.glyphicon-stats:before {
+  content: "\e185";
+}
+.glyphicon-sd-video:before {
+  content: "\e186";
+}
+.glyphicon-hd-video:before {
+  content: "\e187";
+}
+.glyphicon-subtitles:before {
+  content: "\e188";
+}
+.glyphicon-sound-stereo:before {
+  content: "\e189";
+}
+.glyphicon-sound-dolby:before {
+  content: "\e190";
+}
+.glyphicon-sound-5-1:before {
+  content: "\e191";
+}
+.glyphicon-sound-6-1:before {
+  content: "\e192";
+}
+.glyphicon-sound-7-1:before {
+  content: "\e193";
+}
+.glyphicon-copyright-mark:before {
+  content: "\e194";
+}
+.glyphicon-registration-mark:before {
+  content: "\e195";
+}
+.glyphicon-cloud-download:before {
+  content: "\e197";
+}
+.glyphicon-cloud-upload:before {
+  content: "\e198";
+}
+.glyphicon-tree-conifer:before {
+  content: "\e199";
+}
+.glyphicon-tree-deciduous:before {
+  content: "\e200";
+}
+.glyphicon-cd:before {
+  content: "\e201";
+}
+.glyphicon-save-file:before {
+  content: "\e202";
+}
+.glyphicon-open-file:before {
+  content: "\e203";
+}
+.glyphicon-level-up:before {
+  content: "\e204";
+}
+.glyphicon-copy:before {
+  content: "\e205";
+}
+.glyphicon-paste:before {
+  content: "\e206";
+}
+.glyphicon-alert:before {
+  content: "\e209";
+}
+.glyphicon-equalizer:before {
+  content: "\e210";
+}
+.glyphicon-king:before {
+  content: "\e211";
+}
+.glyphicon-queen:before {
+  content: "\e212";
+}
+.glyphicon-pawn:before {
+  content: "\e213";
+}
+.glyphicon-bishop:before {
+  content: "\e214";
+}
+.glyphicon-knight:before {
+  content: "\e215";
+}
+.glyphicon-baby-formula:before {
+  content: "\e216";
+}
+.glyphicon-tent:before {
+  content: "\26fa";
+}
+.glyphicon-blackboard:before {
+  content: "\e218";
+}
+.glyphicon-bed:before {
+  content: "\e219";
+}
+.glyphicon-apple:before {
+  content: "\f8ff";
+}
+.glyphicon-erase:before {
+  content: "\e221";
+}
+.glyphicon-hourglass:before {
+  content: "\231b";
+}
+.glyphicon-lamp:before {
+  content: "\e223";
+}
+.glyphicon-duplicate:before {
+  content: "\e224";
+}
+.glyphicon-piggy-bank:before {
+  content: "\e225";
+}
+.glyphicon-scissors:before {
+  content: "\e226";
+}
+.glyphicon-bitcoin:before {
+  content: "\e227";
+}
+.glyphicon-btc:before {
+  content: "\e227";
+}
+.glyphicon-xbt:before {
+  content: "\e227";
+}
+.glyphicon-yen:before {
+  content: "\00a5";
+}
+.glyphicon-jpy:before {
+  content: "\00a5";
+}
+.glyphicon-ruble:before {
+  content: "\20bd";
+}
+.glyphicon-rub:before {
+  content: "\20bd";
+}
+.glyphicon-scale:before {
+  content: "\e230";
+}
+.glyphicon-ice-lolly:before {
+  content: "\e231";
+}
+.glyphicon-ice-lolly-tasted:before {
+  content: "\e232";
+}
+.glyphicon-education:before {
+  content: "\e233";
+}
+.glyphicon-option-horizontal:before {
+  content: "\e234";
+}
+.glyphicon-option-vertical:before {
+  content: "\e235";
+}
+.glyphicon-menu-hamburger:before {
+  content: "\e236";
+}
+.glyphicon-modal-window:before {
+  content: "\e237";
+}
+.glyphicon-oil:before {
+  content: "\e238";
+}
+.glyphicon-grain:before {
+  content: "\e239";
+}
+.glyphicon-sunglasses:before {
+  content: "\e240";
+}
+.glyphicon-text-size:before {
+  content: "\e241";
+}
+.glyphicon-text-color:before {
+  content: "\e242";
+}
+.glyphicon-text-background:before {
+  content: "\e243";
+}
+.glyphicon-object-align-top:before {
+  content: "\e244";
+}
+.glyphicon-object-align-bottom:before {
+  content: "\e245";
+}
+.glyphicon-object-align-horizontal:before {
+  content: "\e246";
+}
+.glyphicon-object-align-left:before {
+  content: "\e247";
+}
+.glyphicon-object-align-vertical:before {
+  content: "\e248";
+}
+.glyphicon-object-align-right:before {
+  content: "\e249";
+}
+.glyphicon-triangle-right:before {
+  content: "\e250";
+}
+.glyphicon-triangle-left:before {
+  content: "\e251";
+}
+.glyphicon-triangle-bottom:before {
+  content: "\e252";
+}
+.glyphicon-triangle-top:before {
+  content: "\e253";
+}
+.glyphicon-console:before {
+  content: "\e254";
+}
+.glyphicon-superscript:before {
+  content: "\e255";
+}
+.glyphicon-subscript:before {
+  content: "\e256";
+}
+.glyphicon-menu-left:before {
+  content: "\e257";
+}
+.glyphicon-menu-right:before {
+  content: "\e258";
+}
+.glyphicon-menu-down:before {
+  content: "\e259";
+}
+.glyphicon-menu-up:before {
+  content: "\e260";
+}
+* {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+*:before,
+*:after {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+html {
+  font-size: 10px;
+  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
+}
+body {
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #333333;
+  background-color: #fff;
+}
+input,
+button,
+select,
+textarea {
+  font-family: inherit;
+  font-size: inherit;
+  line-height: inherit;
+}
+a {
+  color: #158cba;
+  text-decoration: none;
+}
+a:hover,
+a:focus {
+  color: #0d5875;
+  text-decoration: underline;
+}
+a:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+figure {
+  margin: 0;
+}
+img {
+  vertical-align: middle;
+}
+.img-responsive,
+.thumbnail > img,
+.thumbnail a > img,
+.carousel-inner > .item > img,
+.carousel-inner > .item > a > img {
+  display: block;
+  max-width: 100%;
+  height: auto;
+}
+.img-rounded {
+  border-radius: 6px;
+}
+.img-thumbnail {
+  padding: 4px;
+  line-height: 1.42857143;
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 4px;
+  -webkit-transition: all 0.2s ease-in-out;
+  -o-transition: all 0.2s ease-in-out;
+  transition: all 0.2s ease-in-out;
+  display: inline-block;
+  max-width: 100%;
+  height: auto;
+}
+.img-circle {
+  border-radius: 50%;
+}
+hr {
+  margin-top: 20px;
+  margin-bottom: 20px;
+  border: 0;
+  border-top: 1px solid #eeeeee;
+}
+.sr-only {
+  position: absolute;
+  width: 1px;
+  height: 1px;
+  margin: -1px;
+  padding: 0;
+  overflow: hidden;
+  clip: rect(0, 0, 0, 0);
+  border: 0;
+}
+.sr-only-focusable:active,
+.sr-only-focusable:focus {
+  position: static;
+  width: auto;
+  height: auto;
+  margin: 0;
+  overflow: visible;
+  clip: auto;
+}
+[role="button"] {
+  cursor: pointer;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6,
+.h1,
+.h2,
+.h3,
+.h4,
+.h5,
+.h6 {
+  font-family: inherit;
+  font-weight: 500;
+  line-height: 1.1;
+  color: inherit;
+}
+h1 small,
+h2 small,
+h3 small,
+h4 small,
+h5 small,
+h6 small,
+.h1 small,
+.h2 small,
+.h3 small,
+.h4 small,
+.h5 small,
+.h6 small,
+h1 .small,
+h2 .small,
+h3 .small,
+h4 .small,
+h5 .small,
+h6 .small,
+.h1 .small,
+.h2 .small,
+.h3 .small,
+.h4 .small,
+.h5 .small,
+.h6 .small {
+  font-weight: normal;
+  line-height: 1;
+  color: #777777;
+}
+h1,
+.h1,
+h2,
+.h2,
+h3,
+.h3 {
+  margin-top: 20px;
+  margin-bottom: 10px;
+}
+h1 small,
+.h1 small,
+h2 small,
+.h2 small,
+h3 small,
+.h3 small,
+h1 .small,
+.h1 .small,
+h2 .small,
+.h2 .small,
+h3 .small,
+.h3 .small {
+  font-size: 65%;
+}
+h4,
+.h4,
+h5,
+.h5,
+h6,
+.h6 {
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+h4 small,
+.h4 small,
+h5 small,
+.h5 small,
+h6 small,
+.h6 small,
+h4 .small,
+.h4 .small,
+h5 .small,
+.h5 .small,
+h6 .small,
+.h6 .small {
+  font-size: 75%;
+}
+h1,
+.h1 {
+  font-size: 28px;
+}
+h2,
+.h2 {
+  font-size: 23px;
+}
+h3,
+.h3 {
+  font-size: 21px;
+}
+h4,
+.h4 {
+  font-size: 18px;
+}
+h5,
+.h5 {
+  font-size: 14px;
+}
+h6,
+.h6 {
+  font-size: 12px;
+}
+p {
+  margin: 0 0 10px;
+}
+.lead {
+  margin-bottom: 20px;
+  font-size: 16px;
+  font-weight: 300;
+  line-height: 1.4;
+}
+@media (min-width: 768px) {
+  .lead {
+    font-size: 21px;
+  }
+}
+small,
+.small {
+  font-size: 85%;
+}
+mark,
+.mark {
+  background-color: #fcf8e3;
+  padding: .2em;
+}
+.text-left {
+  text-align: left;
+}
+.text-right {
+  text-align: right;
+}
+.text-center {
+  text-align: center;
+}
+.text-justify {
+  text-align: justify;
+}
+.text-nowrap {
+  white-space: nowrap;
+}
+.text-lowercase {
+  text-transform: lowercase;
+}
+.text-uppercase {
+  text-transform: uppercase;
+}
+.text-capitalize {
+  text-transform: capitalize;
+}
+.text-muted {
+  color: #777777;
+}
+.text-primary {
+  color: #158cba;
+}
+a.text-primary:hover,
+a.text-primary:focus {
+  color: #106a8c;
+}
+.text-success {
+  color: #3c763d;
+}
+a.text-success:hover,
+a.text-success:focus {
+  color: #2b542c;
+}
+.text-info {
+  color: #31708f;
+}
+a.text-info:hover,
+a.text-info:focus {
+  color: #245269;
+}
+.text-warning {
+  color: #8a6d3b;
+}
+a.text-warning:hover,
+a.text-warning:focus {
+  color: #66512c;
+}
+.text-danger {
+  color: #a94442;
+}
+a.text-danger:hover,
+a.text-danger:focus {
+  color: #843534;
+}
+.bg-primary {
+  color: #fff;
+  background-color: #158cba;
+}
+a.bg-primary:hover,
+a.bg-primary:focus {
+  background-color: #106a8c;
+}
+.bg-success {
+  background-color: #dff0d8;
+}
+a.bg-success:hover,
+a.bg-success:focus {
+  background-color: #c1e2b3;
+}
+.bg-info {
+  background-color: #d9edf7;
+}
+a.bg-info:hover,
+a.bg-info:focus {
+  background-color: #afd9ee;
+}
+.bg-warning {
+  background-color: #fcf8e3;
+}
+a.bg-warning:hover,
+a.bg-warning:focus {
+  background-color: #f7ecb5;
+}
+.bg-danger {
+  background-color: #f2dede;
+}
+a.bg-danger:hover,
+a.bg-danger:focus {
+  background-color: #e4b9b9;
+}
+.page-header {
+  padding-bottom: 9px;
+  margin: 40px 0 20px;
+  border-bottom: 1px solid #eeeeee;
+}
+ul,
+ol {
+  margin-top: 0;
+  margin-bottom: 10px;
+}
+ul ul,
+ol ul,
+ul ol,
+ol ol {
+  margin-bottom: 0;
+}
+.list-unstyled {
+  padding-left: 0;
+  list-style: none;
+}
+.list-inline {
+  padding-left: 0;
+  list-style: none;
+  margin-left: -5px;
+}
+.list-inline > li {
+  display: inline-block;
+  padding-left: 5px;
+  padding-right: 5px;
+}
+dl {
+  margin-top: 0;
+  margin-bottom: 20px;
+}
+dt,
+dd {
+  line-height: 1.42857143;
+}
+dt {
+  font-weight: bold;
+}
+dd {
+  margin-left: 0;
+}
+@media (min-width: 768px) {
+  .dl-horizontal dt {
+    float: left;
+    width: 160px;
+    clear: left;
+    text-align: right;
+    overflow: hidden;
+    text-overflow: ellipsis;
+    white-space: nowrap;
+  }
+  .dl-horizontal dd {
+    margin-left: 180px;
+  }
+}
+abbr[title],
+abbr[data-original-title] {
+  cursor: help;
+  border-bottom: 1px dotted #777777;
+}
+.initialism {
+  font-size: 90%;
+  text-transform: uppercase;
+}
+blockquote {
+  padding: 10px 20px;
+  margin: 0 0 20px;
+  font-size: 17.5px;
+  border-left: 5px solid #eeeeee;
+}
+blockquote p:last-child,
+blockquote ul:last-child,
+blockquote ol:last-child {
+  margin-bottom: 0;
+}
+blockquote footer,
+blockquote small,
+blockquote .small {
+  display: block;
+  font-size: 80%;
+  line-height: 1.42857143;
+  color: #777777;
+}
+blockquote footer:before,
+blockquote small:before,
+blockquote .small:before {
+  content: '\2014 \00A0';
+}
+.blockquote-reverse,
+blockquote.pull-right {
+  padding-right: 15px;
+  padding-left: 0;
+  border-right: 5px solid #eeeeee;
+  border-left: 0;
+  text-align: right;
+}
+.blockquote-reverse footer:before,
+blockquote.pull-right footer:before,
+.blockquote-reverse small:before,
+blockquote.pull-right small:before,
+.blockquote-reverse .small:before,
+blockquote.pull-right .small:before {
+  content: '';
+}
+.blockquote-reverse footer:after,
+blockquote.pull-right footer:after,
+.blockquote-reverse small:after,
+blockquote.pull-right small:after,
+.blockquote-reverse .small:after,
+blockquote.pull-right .small:after {
+  content: '\00A0 \2014';
+}
+address {
+  margin-bottom: 20px;
+  font-style: normal;
+  line-height: 1.42857143;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
+}
+code {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #c7254e;
+  background-color: #f9f2f4;
+  border-radius: 4px;
+}
+kbd {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #fff;
+  background-color: #333;
+  border-radius: 3px;
+  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);
+}
+kbd kbd {
+  padding: 0;
+  font-size: 100%;
+  font-weight: bold;
+  box-shadow: none;
+}
+pre {
+  display: block;
+  padding: 9.5px;
+  margin: 0 0 10px;
+  font-size: 13px;
+  line-height: 1.42857143;
+  word-break: break-all;
+  word-wrap: break-word;
+  color: #333333;
+  background-color: #f5f5f5;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+}
+pre code {
+  padding: 0;
+  font-size: inherit;
+  color: inherit;
+  white-space: pre-wrap;
+  background-color: transparent;
+  border-radius: 0;
+}
+.pre-scrollable {
+  max-height: 340px;
+  overflow-y: scroll;
+}
+.container {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+@media (min-width: 768px) {
+  .container {
+    width: 750px;
+  }
+}
+@media (min-width: 992px) {
+  .container {
+    width: 970px;
+  }
+}
+@media (min-width: 1200px) {
+  .container {
+    width: 1170px;
+  }
+}
+.container-fluid {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.row {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
+  position: relative;
+  min-height: 1px;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
+  float: left;
+}
+.col-xs-12 {
+  width: 100%;
+}
+.col-xs-11 {
+  width: 91.66666667%;
+}
+.col-xs-10 {
+  width: 83.33333333%;
+}
+.col-xs-9 {
+  width: 75%;
+}
+.col-xs-8 {
+  width: 66.66666667%;
+}
+.col-xs-7 {
+  width: 58.33333333%;
+}
+.col-xs-6 {
+  width: 50%;
+}
+.col-xs-5 {
+  width: 41.66666667%;
+}
+.col-xs-4 {
+  width: 33.33333333%;
+}
+.col-xs-3 {
+  width: 25%;
+}
+.col-xs-2 {
+  width: 16.66666667%;
+}
+.col-xs-1 {
+  width: 8.33333333%;
+}
+.col-xs-pull-12 {
+  right: 100%;
+}
+.col-xs-pull-11 {
+  right: 91.66666667%;
+}
+.col-xs-pull-10 {
+  right: 83.33333333%;
+}
+.col-xs-pull-9 {
+  right: 75%;
+}
+.col-xs-pull-8 {
+  right: 66.66666667%;
+}
+.col-xs-pull-7 {
+  right: 58.33333333%;
+}
+.col-xs-pull-6 {
+  right: 50%;
+}
+.col-xs-pull-5 {
+  right: 41.66666667%;
+}
+.col-xs-pull-4 {
+  right: 33.33333333%;
+}
+.col-xs-pull-3 {
+  right: 25%;
+}
+.col-xs-pull-2 {
+  right: 16.66666667%;
+}
+.col-xs-pull-1 {
+  right: 8.33333333%;
+}
+.col-xs-pull-0 {
+  right: auto;
+}
+.col-xs-push-12 {
+  left: 100%;
+}
+.col-xs-push-11 {
+  left: 91.66666667%;
+}
+.col-xs-push-10 {
+  left: 83.33333333%;
+}
+.col-xs-push-9 {
+  left: 75%;
+}
+.col-xs-push-8 {
+  left: 66.66666667%;
+}
+.col-xs-push-7 {
+  left: 58.33333333%;
+}
+.col-xs-push-6 {
+  left: 50%;
+}
+.col-xs-push-5 {
+  left: 41.66666667%;
+}
+.col-xs-push-4 {
+  left: 33.33333333%;
+}
+.col-xs-push-3 {
+  left: 25%;
+}
+.col-xs-push-2 {
+  left: 16.66666667%;
+}
+.col-xs-push-1 {
+  left: 8.33333333%;
+}
+.col-xs-push-0 {
+  left: auto;
+}
+.col-xs-offset-12 {
+  margin-left: 100%;
+}
+.col-xs-offset-11 {
+  margin-left: 91.66666667%;
+}
+.col-xs-offset-10 {
+  margin-left: 83.33333333%;
+}
+.col-xs-offset-9 {
+  margin-left: 75%;
+}
+.col-xs-offset-8 {
+  margin-left: 66.66666667%;
+}
+.col-xs-offset-7 {
+  margin-left: 58.33333333%;
+}
+.col-xs-offset-6 {
+  margin-left: 50%;
+}
+.col-xs-offset-5 {
+  margin-left: 41.66666667%;
+}
+.col-xs-offset-4 {
+  margin-left: 33.33333333%;
+}
+.col-xs-offset-3 {
+  margin-left: 25%;
+}
+.col-xs-offset-2 {
+  margin-left: 16.66666667%;
+}
+.col-xs-offset-1 {
+  margin-left: 8.33333333%;
+}
+.col-xs-offset-0 {
+  margin-left: 0%;
+}
+@media (min-width: 768px) {
+  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
+    float: left;
+  }
+  .col-sm-12 {
+    width: 100%;
+  }
+  .col-sm-11 {
+    width: 91.66666667%;
+  }
+  .col-sm-10 {
+    width: 83.33333333%;
+  }
+  .col-sm-9 {
+    width: 75%;
+  }
+  .col-sm-8 {
+    width: 66.66666667%;
+  }
+  .col-sm-7 {
+    width: 58.33333333%;
+  }
+  .col-sm-6 {
+    width: 50%;
+  }
+  .col-sm-5 {
+    width: 41.66666667%;
+  }
+  .col-sm-4 {
+    width: 33.33333333%;
+  }
+  .col-sm-3 {
+    width: 25%;
+  }
+  .col-sm-2 {
+    width: 16.66666667%;
+  }
+  .col-sm-1 {
+    width: 8.33333333%;
+  }
+  .col-sm-pull-12 {
+    right: 100%;
+  }
+  .col-sm-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-sm-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-sm-pull-9 {
+    right: 75%;
+  }
+  .col-sm-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-sm-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-sm-pull-6 {
+    right: 50%;
+  }
+  .col-sm-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-sm-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-sm-pull-3 {
+    right: 25%;
+  }
+  .col-sm-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-sm-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-sm-pull-0 {
+    right: auto;
+  }
+  .col-sm-push-12 {
+    left: 100%;
+  }
+  .col-sm-push-11 {
+    left: 91.66666667%;
+  }
+  .col-sm-push-10 {
+    left: 83.33333333%;
+  }
+  .col-sm-push-9 {
+    left: 75%;
+  }
+  .col-sm-push-8 {
+    left: 66.66666667%;
+  }
+  .col-sm-push-7 {
+    left: 58.33333333%;
+  }
+  .col-sm-push-6 {
+    left: 50%;
+  }
+  .col-sm-push-5 {
+    left: 41.66666667%;
+  }
+  .col-sm-push-4 {
+    left: 33.33333333%;
+  }
+  .col-sm-push-3 {
+    left: 25%;
+  }
+  .col-sm-push-2 {
+    left: 16.66666667%;
+  }
+  .col-sm-push-1 {
+    left: 8.33333333%;
+  }
+  .col-sm-push-0 {
+    left: auto;
+  }
+  .col-sm-offset-12 {
+    margin-left: 100%;
+  }
+  .col-sm-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-sm-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-sm-offset-9 {
+    margin-left: 75%;
+  }
+  .col-sm-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-sm-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-sm-offset-6 {
+    margin-left: 50%;
+  }
+  .col-sm-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-sm-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-sm-offset-3 {
+    margin-left: 25%;
+  }
+  .col-sm-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-sm-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-sm-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 992px) {
+  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
+    float: left;
+  }
+  .col-md-12 {
+    width: 100%;
+  }
+  .col-md-11 {
+    width: 91.66666667%;
+  }
+  .col-md-10 {
+    width: 83.33333333%;
+  }
+  .col-md-9 {
+    width: 75%;
+  }
+  .col-md-8 {
+    width: 66.66666667%;
+  }
+  .col-md-7 {
+    width: 58.33333333%;
+  }
+  .col-md-6 {
+    width: 50%;
+  }
+  .col-md-5 {
+    width: 41.66666667%;
+  }
+  .col-md-4 {
+    width: 33.33333333%;
+  }
+  .col-md-3 {
+    width: 25%;
+  }
+  .col-md-2 {
+    width: 16.66666667%;
+  }
+  .col-md-1 {
+    width: 8.33333333%;
+  }
+  .col-md-pull-12 {
+    right: 100%;
+  }
+  .col-md-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-md-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-md-pull-9 {
+    right: 75%;
+  }
+  .col-md-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-md-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-md-pull-6 {
+    right: 50%;
+  }
+  .col-md-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-md-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-md-pull-3 {
+    right: 25%;
+  }
+  .col-md-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-md-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-md-pull-0 {
+    right: auto;
+  }
+  .col-md-push-12 {
+    left: 100%;
+  }
+  .col-md-push-11 {
+    left: 91.66666667%;
+  }
+  .col-md-push-10 {
+    left: 83.33333333%;
+  }
+  .col-md-push-9 {
+    left: 75%;
+  }
+  .col-md-push-8 {
+    left: 66.66666667%;
+  }
+  .col-md-push-7 {
+    left: 58.33333333%;
+  }
+  .col-md-push-6 {
+    left: 50%;
+  }
+  .col-md-push-5 {
+    left: 41.66666667%;
+  }
+  .col-md-push-4 {
+    left: 33.33333333%;
+  }
+  .col-md-push-3 {
+    left: 25%;
+  }
+  .col-md-push-2 {
+    left: 16.66666667%;
+  }
+  .col-md-push-1 {
+    left: 8.33333333%;
+  }
+  .col-md-push-0 {
+    left: auto;
+  }
+  .col-md-offset-12 {
+    margin-left: 100%;
+  }
+  .col-md-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-md-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-md-offset-9 {
+    margin-left: 75%;
+  }
+  .col-md-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-md-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-md-offset-6 {
+    margin-left: 50%;
+  }
+  .col-md-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-md-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-md-offset-3 {
+    margin-left: 25%;
+  }
+  .col-md-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-md-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-md-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 1200px) {
+  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
+    float: left;
+  }
+  .col-lg-12 {
+    width: 100%;
+  }
+  .col-lg-11 {
+    width: 91.66666667%;
+  }
+  .col-lg-10 {
+    width: 83.33333333%;
+  }
+  .col-lg-9 {
+    width: 75%;
+  }
+  .col-lg-8 {
+    width: 66.66666667%;
+  }
+  .col-lg-7 {
+    width: 58.33333333%;
+  }
+  .col-lg-6 {
+    width: 50%;
+  }
+  .col-lg-5 {
+    width: 41.66666667%;
+  }
+  .col-lg-4 {
+    width: 33.33333333%;
+  }
+  .col-lg-3 {
+    width: 25%;
+  }
+  .col-lg-2 {
+    width: 16.66666667%;
+  }
+  .col-lg-1 {
+    width: 8.33333333%;
+  }
+  .col-lg-pull-12 {
+    right: 100%;
+  }
+  .col-lg-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-lg-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-lg-pull-9 {
+    right: 75%;
+  }
+  .col-lg-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-lg-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-lg-pull-6 {
+    right: 50%;
+  }
+  .col-lg-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-lg-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-lg-pull-3 {
+    right: 25%;
+  }
+  .col-lg-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-lg-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-lg-pull-0 {
+    right: auto;
+  }
+  .col-lg-push-12 {
+    left: 100%;
+  }
+  .col-lg-push-11 {
+    left: 91.66666667%;
+  }
+  .col-lg-push-10 {
+    left: 83.33333333%;
+  }
+  .col-lg-push-9 {
+    left: 75%;
+  }
+  .col-lg-push-8 {
+    left: 66.66666667%;
+  }
+  .col-lg-push-7 {
+    left: 58.33333333%;
+  }
+  .col-lg-push-6 {
+    left: 50%;
+  }
+  .col-lg-push-5 {
+    left: 41.66666667%;
+  }
+  .col-lg-push-4 {
+    left: 33.33333333%;
+  }
+  .col-lg-push-3 {
+    left: 25%;
+  }
+  .col-lg-push-2 {
+    left: 16.66666667%;
+  }
+  .col-lg-push-1 {
+    left: 8.33333333%;
+  }
+  .col-lg-push-0 {
+    left: auto;
+  }
+  .col-lg-offset-12 {
+    margin-left: 100%;
+  }
+  .col-lg-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-lg-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-lg-offset-9 {
+    margin-left: 75%;
+  }
+  .col-lg-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-lg-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-lg-offset-6 {
+    margin-left: 50%;
+  }
+  .col-lg-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-lg-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-lg-offset-3 {
+    margin-left: 25%;
+  }
+  .col-lg-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-lg-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-lg-offset-0 {
+    margin-left: 0%;
+  }
+}
+table {
+  background-color: transparent;
+}
+caption {
+  padding-top: 8px;
+  padding-bottom: 8px;
+  color: #777777;
+  text-align: left;
+}
+th {
+  text-align: left;
+}
+.table {
+  width: 100%;
+  max-width: 100%;
+  margin-bottom: 20px;
+}
+.table > thead > tr > th,
+.table > tbody > tr > th,
+.table > tfoot > tr > th,
+.table > thead > tr > td,
+.table > tbody > tr > td,
+.table > tfoot > tr > td {
+  padding: 8px;
+  line-height: 1.42857143;
+  vertical-align: top;
+  border-top: 1px solid #ddd;
+}
+.table > thead > tr > th {
+  vertical-align: bottom;
+  border-bottom: 2px solid #ddd;
+}
+.table > caption + thead > tr:first-child > th,
+.table > colgroup + thead > tr:first-child > th,
+.table > thead:first-child > tr:first-child > th,
+.table > caption + thead > tr:first-child > td,
+.table > colgroup + thead > tr:first-child > td,
+.table > thead:first-child > tr:first-child > td {
+  border-top: 0;
+}
+.table > tbody + tbody {
+  border-top: 2px solid #ddd;
+}
+.table .table {
+  background-color: #fff;
+}
+.table-condensed > thead > tr > th,
+.table-condensed > tbody > tr > th,
+.table-condensed > tfoot > tr > th,
+.table-condensed > thead > tr > td,
+.table-condensed > tbody > tr > td,
+.table-condensed > tfoot > tr > td {
+  padding: 5px;
+}
+.table-bordered {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > tbody > tr > th,
+.table-bordered > tfoot > tr > th,
+.table-bordered > thead > tr > td,
+.table-bordered > tbody > tr > td,
+.table-bordered > tfoot > tr > td {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > thead > tr > td {
+  border-bottom-width: 2px;
+}
+.table-striped > tbody > tr:nth-of-type(odd) {
+  background-color: #f9f9f9;
+}
+.table-hover > tbody > tr:hover {
+  background-color: #f5f5f5;
+}
+table col[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-column;
+}
+table td[class*="col-"],
+table th[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-cell;
+}
+.table > thead > tr > td.active,
+.table > tbody > tr > td.active,
+.table > tfoot > tr > td.active,
+.table > thead > tr > th.active,
+.table > tbody > tr > th.active,
+.table > tfoot > tr > th.active,
+.table > thead > tr.active > td,
+.table > tbody > tr.active > td,
+.table > tfoot > tr.active > td,
+.table > thead > tr.active > th,
+.table > tbody > tr.active > th,
+.table > tfoot > tr.active > th {
+  background-color: #f5f5f5;
+}
+.table-hover > tbody > tr > td.active:hover,
+.table-hover > tbody > tr > th.active:hover,
+.table-hover > tbody > tr.active:hover > td,
+.table-hover > tbody > tr:hover > .active,
+.table-hover > tbody > tr.active:hover > th {
+  background-color: #e8e8e8;
+}
+.table > thead > tr > td.success,
+.table > tbody > tr > td.success,
+.table > tfoot > tr > td.success,
+.table > thead > tr > th.success,
+.table > tbody > tr > th.success,
+.table > tfoot > tr > th.success,
+.table > thead > tr.success > td,
+.table > tbody > tr.success > td,
+.table > tfoot > tr.success > td,
+.table > thead > tr.success > th,
+.table > tbody > tr.success > th,
+.table > tfoot > tr.success > th {
+  background-color: #dff0d8;
+}
+.table-hover > tbody > tr > td.success:hover,
+.table-hover > tbody > tr > th.success:hover,
+.table-hover > tbody > tr.success:hover > td,
+.table-hover > tbody > tr:hover > .success,
+.table-hover > tbody > tr.success:hover > th {
+  background-color: #d0e9c6;
+}
+.table > thead > tr > td.info,
+.table > tbody > tr > td.info,
+.table > tfoot > tr > td.info,
+.table > thead > tr > th.info,
+.table > tbody > tr > th.info,
+.table > tfoot > tr > th.info,
+.table > thead > tr.info > td,
+.table > tbody > tr.info > td,
+.table > tfoot > tr.info > td,
+.table > thead > tr.info > th,
+.table > tbody > tr.info > th,
+.table > tfoot > tr.info > th {
+  background-color: #d9edf7;
+}
+.table-hover > tbody > tr > td.info:hover,
+.table-hover > tbody > tr > th.info:hover,
+.table-hover > tbody > tr.info:hover > td,
+.table-hover > tbody > tr:hover > .info,
+.table-hover > tbody > tr.info:hover > th {
+  background-color: #c4e3f3;
+}
+.table > thead > tr > td.warning,
+.table > tbody > tr > td.warning,
+.table > tfoot > tr > td.warning,
+.table > thead > tr > th.warning,
+.table > tbody > tr > th.warning,
+.table > tfoot > tr > th.warning,
+.table > thead > tr.warning > td,
+.table > tbody > tr.warning > td,
+.table > tfoot > tr.warning > td,
+.table > thead > tr.warning > th,
+.table > tbody > tr.warning > th,
+.table > tfoot > tr.warning > th {
+  background-color: #fcf8e3;
+}
+.table-hover > tbody > tr > td.warning:hover,
+.table-hover > tbody > tr > th.warning:hover,
+.table-hover > tbody > tr.warning:hover > td,
+.table-hover > tbody > tr:hover > .warning,
+.table-hover > tbody > tr.warning:hover > th {
+  background-color: #faf2cc;
+}
+.table > thead > tr > td.danger,
+.table > tbody > tr > td.danger,
+.table > tfoot > tr > td.danger,
+.table > thead > tr > th.danger,
+.table > tbody > tr > th.danger,
+.table > tfoot > tr > th.danger,
+.table > thead > tr.danger > td,
+.table > tbody > tr.danger > td,
+.table > tfoot > tr.danger > td,
+.table > thead > tr.danger > th,
+.table > tbody > tr.danger > th,
+.table > tfoot > tr.danger > th {
+  background-color: #f2dede;
+}
+.table-hover > tbody > tr > td.danger:hover,
+.table-hover > tbody > tr > th.danger:hover,
+.table-hover > tbody > tr.danger:hover > td,
+.table-hover > tbody > tr:hover > .danger,
+.table-hover > tbody > tr.danger:hover > th {
+  background-color: #ebcccc;
+}
+.table-responsive {
+  overflow-x: auto;
+  min-height: 0.01%;
+}
+@media screen and (max-width: 767px) {
+  .table-responsive {
+    width: 100%;
+    margin-bottom: 15px;
+    overflow-y: hidden;
+    -ms-overflow-style: -ms-autohiding-scrollbar;
+    border: 1px solid #ddd;
+  }
+  .table-responsive > .table {
+    margin-bottom: 0;
+  }
+  .table-responsive > .table > thead > tr > th,
+  .table-responsive > .table > tbody > tr > th,
+  .table-responsive > .table > tfoot > tr > th,
+  .table-responsive > .table > thead > tr > td,
+  .table-responsive > .table > tbody > tr > td,
+  .table-responsive > .table > tfoot > tr > td {
+    white-space: nowrap;
+  }
+  .table-responsive > .table-bordered {
+    border: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:first-child,
+  .table-responsive > .table-bordered > tbody > tr > th:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
+  .table-responsive > .table-bordered > thead > tr > td:first-child,
+  .table-responsive > .table-bordered > tbody > tr > td:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
+    border-left: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:last-child,
+  .table-responsive > .table-bordered > tbody > tr > th:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
+  .table-responsive > .table-bordered > thead > tr > td:last-child,
+  .table-responsive > .table-bordered > tbody > tr > td:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
+    border-right: 0;
+  }
+  .table-responsive > .table-bordered > tbody > tr:last-child > th,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
+  .table-responsive > .table-bordered > tbody > tr:last-child > td,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
+    border-bottom: 0;
+  }
+}
+fieldset {
+  padding: 0;
+  margin: 0;
+  border: 0;
+  min-width: 0;
+}
+legend {
+  display: block;
+  width: 100%;
+  padding: 0;
+  margin-bottom: 20px;
+  font-size: 21px;
+  line-height: inherit;
+  color: #333333;
+  border: 0;
+  border-bottom: 1px solid #e5e5e5;
+}
+label {
+  display: inline-block;
+  max-width: 100%;
+  margin-bottom: 5px;
+  font-weight: bold;
+}
+input[type="search"] {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+input[type="radio"],
+input[type="checkbox"] {
+  margin: 4px 0 0;
+  margin-top: 1px \9;
+  line-height: normal;
+}
+input[type="file"] {
+  display: block;
+}
+input[type="range"] {
+  display: block;
+  width: 100%;
+}
+select[multiple],
+select[size] {
+  height: auto;
+}
+input[type="file"]:focus,
+input[type="radio"]:focus,
+input[type="checkbox"]:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+output {
+  display: block;
+  padding-top: 7px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555555;
+}
+.form-control {
+  display: block;
+  width: 100%;
+  height: 34px;
+  padding: 6px 12px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555555;
+  background-color: #fff;
+  background-image: none;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+  -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+  transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+}
+.form-control:focus {
+  border-color: #66afe9;
+  outline: 0;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+  box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+}
+.form-control::-moz-placeholder {
+  color: #999;
+  opacity: 1;
+}
+.form-control:-ms-input-placeholder {
+  color: #999;
+}
+.form-control::-webkit-input-placeholder {
+  color: #999;
+}
+.form-control[disabled],
+.form-control[readonly],
+fieldset[disabled] .form-control {
+  background-color: #eeeeee;
+  opacity: 1;
+}
+.form-control[disabled],
+fieldset[disabled] .form-control {
+  cursor: not-allowed;
+}
+textarea.form-control {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: none;
+}
+@media screen and (-webkit-min-device-pixel-ratio: 0) {
+  input[type="date"].form-control,
+  input[type="time"].form-control,
+  input[type="datetime-local"].form-control,
+  input[type="month"].form-control {
+    line-height: 34px;
+  }
+  input[type="date"].input-sm,
+  input[type="time"].input-sm,
+  input[type="datetime-local"].input-sm,
+  input[type="month"].input-sm,
+  .input-group-sm input[type="date"],
+  .input-group-sm input[type="time"],
+  .input-group-sm input[type="datetime-local"],
+  .input-group-sm input[type="month"] {
+    line-height: 30px;
+  }
+  input[type="date"].input-lg,
+  input[type="time"].input-lg,
+  input[type="datetime-local"].input-lg,
+  input[type="month"].input-lg,
+  .input-group-lg input[type="date"],
+  .input-group-lg input[type="time"],
+  .input-group-lg input[type="datetime-local"],
+  .input-group-lg input[type="month"] {
+    line-height: 46px;
+  }
+}
+.form-group {
+  margin-bottom: 15px;
+}
+.radio,
+.checkbox {
+  position: relative;
+  display: block;
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+.radio label,
+.checkbox label {
+  min-height: 20px;
+  padding-left: 20px;
+  margin-bottom: 0;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio input[type="radio"],
+.radio-inline input[type="radio"],
+.checkbox input[type="checkbox"],
+.checkbox-inline input[type="checkbox"] {
+  position: absolute;
+  margin-left: -20px;
+  margin-top: 4px \9;
+}
+.radio + .radio,
+.checkbox + .checkbox {
+  margin-top: -5px;
+}
+.radio-inline,
+.checkbox-inline {
+  position: relative;
+  display: inline-block;
+  padding-left: 20px;
+  margin-bottom: 0;
+  vertical-align: middle;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio-inline + .radio-inline,
+.checkbox-inline + .checkbox-inline {
+  margin-top: 0;
+  margin-left: 10px;
+}
+input[type="radio"][disabled],
+input[type="checkbox"][disabled],
+input[type="radio"].disabled,
+input[type="checkbox"].disabled,
+fieldset[disabled] input[type="radio"],
+fieldset[disabled] input[type="checkbox"] {
+  cursor: not-allowed;
+}
+.radio-inline.disabled,
+.checkbox-inline.disabled,
+fieldset[disabled] .radio-inline,
+fieldset[disabled] .checkbox-inline {
+  cursor: not-allowed;
+}
+.radio.disabled label,
+.checkbox.disabled label,
+fieldset[disabled] .radio label,
+fieldset[disabled] .checkbox label {
+  cursor: not-allowed;
+}
+.form-control-static {
+  padding-top: 7px;
+  padding-bottom: 7px;
+  margin-bottom: 0;
+  min-height: 34px;
+}
+.form-control-static.input-lg,
+.form-control-static.input-sm {
+  padding-left: 0;
+  padding-right: 0;
+}
+.input-sm {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+select.input-sm {
+  height: 30px;
+  line-height: 30px;
+}
+textarea.input-sm,
+select[multiple].input-sm {
+  height: auto;
+}
+.form-group-sm .form-control {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+.form-group-sm select.form-control {
+  height: 30px;
+  line-height: 30px;
+}
+.form-group-sm textarea.form-control,
+.form-group-sm select[multiple].form-control {
+  height: auto;
+}
+.form-group-sm .form-control-static {
+  height: 30px;
+  min-height: 32px;
+  padding: 6px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+}
+.input-lg {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+  border-radius: 6px;
+}
+select.input-lg {
+  height: 46px;
+  line-height: 46px;
+}
+textarea.input-lg,
+select[multiple].input-lg {
+  height: auto;
+}
+.form-group-lg .form-control {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+  border-radius: 6px;
+}
+.form-group-lg select.form-control {
+  height: 46px;
+  line-height: 46px;
+}
+.form-group-lg textarea.form-control,
+.form-group-lg select[multiple].form-control {
+  height: auto;
+}
+.form-group-lg .form-control-static {
+  height: 46px;
+  min-height: 38px;
+  padding: 11px 16px;
+  font-size: 18px;
+  line-height: 1.3333333;
+}
+.has-feedback {
+  position: relative;
+}
+.has-feedback .form-control {
+  padding-right: 42.5px;
+}
+.form-control-feedback {
+  position: absolute;
+  top: 0;
+  right: 0;
+  z-index: 2;
+  display: block;
+  width: 34px;
+  height: 34px;
+  line-height: 34px;
+  text-align: center;
+  pointer-events: none;
+}
+.input-lg + .form-control-feedback,
+.input-group-lg + .form-control-feedback,
+.form-group-lg .form-control + .form-control-feedback {
+  width: 46px;
+  height: 46px;
+  line-height: 46px;
+}
+.input-sm + .form-control-feedback,
+.input-group-sm + .form-control-feedback,
+.form-group-sm .form-control + .form-control-feedback {
+  width: 30px;
+  height: 30px;
+  line-height: 30px;
+}
+.has-success .help-block,
+.has-success .control-label,
+.has-success .radio,
+.has-success .checkbox,
+.has-success .radio-inline,
+.has-success .checkbox-inline,
+.has-success.radio label,
+.has-success.checkbox label,
+.has-success.radio-inline label,
+.has-success.checkbox-inline label {
+  color: #3c763d;
+}
+.has-success .form-control {
+  border-color: #3c763d;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-success .form-control:focus {
+  border-color: #2b542c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;
+}
+.has-success .input-group-addon {
+  color: #3c763d;
+  border-color: #3c763d;
+  background-color: #dff0d8;
+}
+.has-success .form-control-feedback {
+  color: #3c763d;
+}
+.has-warning .help-block,
+.has-warning .control-label,
+.has-warning .radio,
+.has-warning .checkbox,
+.has-warning .radio-inline,
+.has-warning .checkbox-inline,
+.has-warning.radio label,
+.has-warning.checkbox label,
+.has-warning.radio-inline label,
+.has-warning.checkbox-inline label {
+  color: #8a6d3b;
+}
+.has-warning .form-control {
+  border-color: #8a6d3b;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-warning .form-control:focus {
+  border-color: #66512c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;
+}
+.has-warning .input-group-addon {
+  color: #8a6d3b;
+  border-color: #8a6d3b;
+  background-color: #fcf8e3;
+}
+.has-warning .form-control-feedback {
+  color: #8a6d3b;
+}
+.has-error .help-block,
+.has-error .control-label,
+.has-error .radio,
+.has-error .checkbox,
+.has-error .radio-inline,
+.has-error .checkbox-inline,
+.has-error.radio label,
+.has-error.checkbox label,
+.has-error.radio-inline label,
+.has-error.checkbox-inline label {
+  color: #a94442;
+}
+.has-error .form-control {
+  border-color: #a94442;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-error .form-control:focus {
+  border-color: #843534;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;
+}
+.has-error .input-group-addon {
+  color: #a94442;
+  border-color: #a94442;
+  background-color: #f2dede;
+}
+.has-error .form-control-feedback {
+  color: #a94442;
+}
+.has-feedback label ~ .form-control-feedback {
+  top: 25px;
+}
+.has-feedback label.sr-only ~ .form-control-feedback {
+  top: 0;
+}
+.help-block {
+  display: block;
+  margin-top: 5px;
+  margin-bottom: 10px;
+  color: #737373;
+}
+@media (min-width: 768px) {
+  .form-inline .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .form-inline .form-control-static {
+    display: inline-block;
+  }
+  .form-inline .input-group {
+    display: inline-table;
+    vertical-align: middle;
+  }
+  .form-inline .input-group .input-group-addon,
+  .form-inline .input-group .input-group-btn,
+  .form-inline .input-group .form-control {
+    width: auto;
+  }
+  .form-inline .input-group > .form-control {
+    width: 100%;
+  }
+  .form-inline .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio,
+  .form-inline .checkbox {
+    display: inline-block;
+    margin-top: 0;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio label,
+  .form-inline .checkbox label {
+    padding-left: 0;
+  }
+  .form-inline .radio input[type="radio"],
+  .form-inline .checkbox input[type="checkbox"] {
+    position: relative;
+    margin-left: 0;
+  }
+  .form-inline .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox,
+.form-horizontal .radio-inline,
+.form-horizontal .checkbox-inline {
+  margin-top: 0;
+  margin-bottom: 0;
+  padding-top: 7px;
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox {
+  min-height: 27px;
+}
+.form-horizontal .form-group {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .control-label {
+    text-align: right;
+    margin-bottom: 0;
+    padding-top: 7px;
+  }
+}
+.form-horizontal .has-feedback .form-control-feedback {
+  right: 15px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .form-group-lg .control-label {
+    padding-top: 14.333333px;
+    font-size: 18px;
+  }
+}
+@media (min-width: 768px) {
+  .form-horizontal .form-group-sm .control-label {
+    padding-top: 6px;
+    font-size: 12px;
+  }
+}
+.btn {
+  display: inline-block;
+  margin-bottom: 0;
+  font-weight: normal;
+  text-align: center;
+  vertical-align: middle;
+  touch-action: manipulation;
+  cursor: pointer;
+  background-image: none;
+  border: 1px solid transparent;
+  whi

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/flink/blob/a6890b28/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf b/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf
new file mode 100644
index 0000000..f7936cc
Binary files /dev/null and b/flink-runtime-web/web-dashboard/web/fonts/FontAwesome.otf differ


[06/47] flink git commit: [FLINK-2354] [runtime] Add job graph and checkpoint recovery

Posted by tr...@apache.org.
[FLINK-2354] [runtime] Add job graph and checkpoint recovery

This closes #1153.


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

Branch: refs/heads/master
Commit: 73c73e92750ab8fb068d0a3cb37afcb642084fc0
Parents: 3aaee1e
Author: Ufuk Celebi <uc...@apache.org>
Authored: Tue Sep 1 17:25:46 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Oct 20 00:16:51 2015 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |  21 +
 .../checkpoint/CheckpointCoordinator.java       | 223 +++---
 .../runtime/checkpoint/CheckpointIDCounter.java |  43 ++
 .../checkpoint/CheckpointRecoveryFactory.java   |  61 ++
 .../runtime/checkpoint/CompletedCheckpoint.java |  81 +++
 .../checkpoint/CompletedCheckpointStore.java    |  69 ++
 .../runtime/checkpoint/PendingCheckpoint.java   |   6 +-
 .../StandaloneCheckpointIDCounter.java          |  47 ++
 .../StandaloneCheckpointRecoveryFactory.java    |  52 ++
 .../StandaloneCompletedCheckpointStore.java     | 100 +++
 .../flink/runtime/checkpoint/StateForTask.java  |  21 +-
 .../checkpoint/SuccessfulCheckpoint.java        |  82 ---
 .../ZooKeeperCheckpointIDCounter.java           | 130 ++++
 .../ZooKeeperCheckpointRecoveryFactory.java     |  66 ++
 .../ZooKeeperCompletedCheckpointStore.java      | 293 ++++++++
 .../runtime/executiongraph/ExecutionGraph.java  |  51 +-
 .../apache/flink/runtime/jobgraph/JobGraph.java |   5 +
 .../flink/runtime/jobmanager/RecoveryMode.java  |  13 +
 .../StandaloneSubmittedJobGraphStore.java       |  65 ++
 .../runtime/jobmanager/SubmittedJobGraph.java   |  77 ++
 .../jobmanager/SubmittedJobGraphStore.java      |  93 +++
 .../ZooKeeperSubmittedJobGraphStore.java        | 379 ++++++++++
 .../leaderelection/LeaderElectionService.java   |   1 +
 .../ZooKeeperLeaderElectionService.java         |   1 +
 .../flink/runtime/state/StateBackend.java       |  39 +
 .../state/StateHandleProviderFactory.java       |  61 ++
 .../flink/runtime/util/LeaderElectionUtils.java |  67 --
 .../flink/runtime/util/ZooKeeperUtils.java      | 138 +++-
 .../zookeeper/ZooKeeperStateHandleStore.java    | 384 ++++++++++
 .../flink/runtime/jobmanager/JobInfo.scala      |  25 +-
 .../flink/runtime/jobmanager/JobManager.scala   | 594 ++++++++++-----
 .../runtime/messages/JobManagerMessages.scala   |  16 +
 .../runtime/minicluster/FlinkMiniCluster.scala  |   4 +-
 .../checkpoint/CheckpointCoordinatorTest.java   | 190 ++---
 .../checkpoint/CheckpointIDCounterTest.java     | 194 +++++
 .../checkpoint/CheckpointStateRestoreTest.java  |  19 +-
 .../CompletedCheckpointStoreTest.java           | 297 ++++++++
 .../StandaloneCompletedCheckpointStoreTest.java |  33 +
 ...ZooKeeperCompletedCheckpointStoreITCase.java | 101 +++
 .../BlobLibraryCacheManagerTest.java            |   4 +
 .../PartitionRequestClientFactoryTest.java      |   2 +-
 ...ManagerSubmittedJobGraphsRecoveryITCase.java | 460 ++++++++++++
 .../StandaloneSubmittedJobGraphStoreTest.java   |  53 ++
 .../ZooKeeperSubmittedJobGraphsStoreITCase.java | 283 ++++++++
 .../JobManagerLeaderElectionTest.java           |  27 +-
 .../TestingLeaderElectionService.java           |   4 +
 .../ZooKeeperLeaderRetrievalTest.java           |  21 +-
 .../messages/CheckpointMessagesTest.java        |   2 +-
 .../runtime/taskmanager/TaskCancelTest.java     |  67 +-
 .../runtime/testutils/CommonTestUtils.java      |  75 +-
 .../testutils/JobManagerActorTestUtils.java     | 166 +++++
 .../runtime/testutils/JobManagerProcess.java    | 226 ++++++
 .../runtime/testutils/TaskManagerProcess.java   | 133 ++++
 .../flink/runtime/testutils/TestJvmProcess.java | 267 +++++++
 .../runtime/testutils/ZooKeeperTestUtils.java   |  94 +++
 .../ZooKeeperStateHandleStoreITCase.java        | 591 +++++++++++++++
 .../zookeeper/ZooKeeperTestEnvironment.java     | 133 ++++
 .../ExecutionGraphRestartTest.scala             |  21 +-
 .../runtime/testingUtils/TestingCluster.scala   |  44 +-
 .../testingUtils/TestingJobManager.scala        |  32 +-
 .../streaming/runtime/tasks/StreamTask.java     |   2 +-
 .../test/util/ForkableFlinkMiniCluster.scala    |  36 +-
 .../checkpointing/StateCheckpoinedITCase.java   | 391 ----------
 .../checkpointing/StateCheckpointedITCase.java  | 391 ++++++++++
 ...tJobManagerProcessFailureRecoveryITCase.java | 289 ++++++++
 .../AbstractProcessFailureRecoveryTest.java     | 444 ------------
 ...ctTaskManagerProcessFailureRecoveryTest.java | 397 +++++++++++
 .../flink/test/recovery/ChaosMonkeyITCase.java  | 713 +++++++++++++++++++
 .../JobManagerCheckpointRecoveryITCase.java     | 395 ++++++++++
 ...anagerProcessFailureBatchRecoveryITCase.java | 140 ++++
 .../ProcessFailureBatchRecoveryITCase.java      | 115 ---
 .../recovery/ProcessFailureCancelingITCase.java |   4 +-
 .../ProcessFailureStreamingRecoveryITCase.java  | 234 ------
 ...anagerProcessFailureBatchRecoveryITCase.java | 115 +++
 ...erProcessFailureStreamingRecoveryITCase.java | 234 ++++++
 .../ZooKeeperLeaderElectionITCase.java          |  61 +-
 .../org/apache/flink/yarn/YarnJobManager.scala  |  11 +-
 77 files changed, 8901 insertions(+), 1918 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 5d6f1c7..be730a0 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -407,6 +407,12 @@ public final class ConfigConstants {
 	 */
 	public static final String STATE_BACKEND = "state.backend";
 	
+	/**
+	 * File system state backend base path for recoverable state handles. Recovery state is written
+	 * to this path and the file state handles are persisted for recovery.
+	 */
+	public static final String STATE_BACKEND_FS_RECOVERY_PATH = "state.backend.fs.dir.recovery";
+	
 	// ----------------------------- Miscellaneous ----------------------------
 	
 	/**
@@ -433,6 +439,15 @@ public final class ConfigConstants {
 
 	public static final String ZOOKEEPER_LEADER_PATH = "ha.zookeeper.dir.leader";
 
+	/** ZooKeeper root path (ZNode) for job graphs. */
+	public static final String ZOOKEEPER_JOBGRAPHS_PATH = "ha.zookeeper.dir.jobgraphs";
+
+	/** ZooKeeper root path (ZNode) for completed checkpoints. */
+	public static final String ZOOKEEPER_CHECKPOINTS_PATH = "ha.zookeeper.dir.checkpoints";
+
+	/** ZooKeeper root path (ZNode) for checkpoint counters. */
+	public static final String ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "ha.zookeeper.dir.checkpoint-counter";
+
 	public static final String ZOOKEEPER_SESSION_TIMEOUT = "ha.zookeeper.client.session-timeout";
 
 	public static final String ZOOKEEPER_CONNECTION_TIMEOUT = "ha.zookeeper.client.connection-timeout";
@@ -699,6 +714,12 @@ public final class ConfigConstants {
 
 	public static final String DEFAULT_ZOOKEEPER_LEADER_PATH = "/leader";
 
+	public static final String DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH = "/jobgraphs";
+
+	public static final String DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH = "/checkpoints";
+
+	public static final String DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "/checkpoint-counter";
+
 	public static final int DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = 60000;
 
 	public static final int DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT = 15000;

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 8f0b19b..fdb59d9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -21,16 +21,16 @@ package org.apache.flink.runtime.checkpoint;
 import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
-
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete;
 import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint;
@@ -38,7 +38,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayDeque;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -48,13 +47,19 @@ import java.util.Timer;
 import java.util.TimerTask;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
  * The checkpoint coordinator coordinates the distributed snapshots of operators and state.
  * It triggers the checkpoint by sending the messages to the relevant tasks and collects the
  * checkpoint acknowledgements. It also collects and maintains the overview of the state handles
  * reported by the tasks that acknowledge the checkpoint.
+ *
+ * <p>Depending on the configured {@link RecoveryMode}, the behaviour of the {@link
+ * CompletedCheckpointStore} and {@link CheckpointIDCounter} change. The default standalone
+ * implementations don't support any recovery.
  */
 public class CheckpointCoordinator {
 	
@@ -79,12 +84,20 @@ public class CheckpointCoordinator {
 	private final ExecutionVertex[] tasksToCommitTo;
 
 	private final Map<Long, PendingCheckpoint> pendingCheckpoints;
-	
-	private final ArrayDeque<SuccessfulCheckpoint> completedCheckpoints;
+
+	/**
+	 * Completed checkpoints. Implementations can be blocking. Make sure calls to methods
+	 * accessing this don't block the job manager actor and run asynchronously.
+	 */
+	private final CompletedCheckpointStore completedCheckpointStore;
 	
 	private final ArrayDeque<Long> recentPendingCheckpoints;
 
-	private final AtomicLong checkpointIdCounter = new AtomicLong(1);
+	/**
+	 * Checkpoint ID counter to ensure ascending IDs. In case of job manager failures, these
+	 * need to be ascending across job managers.
+	 */
+	private final CheckpointIDCounter checkpointIdCounter;
 
 	private final AtomicInteger numUnsuccessfulCheckpointsTriggers = new AtomicInteger();
 
@@ -93,8 +106,6 @@ public class CheckpointCoordinator {
 	
 	private final long checkpointTimeout;
 	
-	private final int numSuccessfulCheckpointsToRetain;
-	
 	private TimerTask periodicScheduler;
 	
 	private ActorGateway jobStatusListener;
@@ -110,61 +121,62 @@ public class CheckpointCoordinator {
 
 	public CheckpointCoordinator(
 			JobID job,
-			int numSuccessfulCheckpointsToRetain,
 			long checkpointTimeout,
 			ExecutionVertex[] tasksToTrigger,
 			ExecutionVertex[] tasksToWaitFor,
 			ExecutionVertex[] tasksToCommitTo,
-			ClassLoader userClassLoader) {
+			ClassLoader userClassLoader,
+			CheckpointIDCounter checkpointIDCounter,
+			CompletedCheckpointStore completedCheckpointStore,
+			RecoveryMode recoveryMode) throws Exception {
 		
-		// some sanity checks
-		if (job == null || tasksToTrigger == null ||
-				tasksToWaitFor == null || tasksToCommitTo == null) {
-			throw new NullPointerException();
-		}
-		if (numSuccessfulCheckpointsToRetain < 1) {
-			throw new IllegalArgumentException("Must retain at least one successful checkpoint");
-		}
-		if (checkpointTimeout < 1) {
-			throw new IllegalArgumentException("Checkpoint timeout must be larger than zero");
-		}
+		// Sanity check
+		checkArgument(checkpointTimeout >= 1, "Checkpoint timeout must be larger than zero");
 		
-		this.job = job;
-		this.numSuccessfulCheckpointsToRetain = numSuccessfulCheckpointsToRetain;
+		this.job = checkNotNull(job);
 		this.checkpointTimeout = checkpointTimeout;
-		this.tasksToTrigger = tasksToTrigger;
-		this.tasksToWaitFor = tasksToWaitFor;
-		this.tasksToCommitTo = tasksToCommitTo;
+		this.tasksToTrigger = checkNotNull(tasksToTrigger);
+		this.tasksToWaitFor = checkNotNull(tasksToWaitFor);
+		this.tasksToCommitTo = checkNotNull(tasksToCommitTo);
 		this.pendingCheckpoints = new LinkedHashMap<Long, PendingCheckpoint>();
-		this.completedCheckpoints = new ArrayDeque<SuccessfulCheckpoint>(numSuccessfulCheckpointsToRetain + 1);
+		this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
 		this.recentPendingCheckpoints = new ArrayDeque<Long>(NUM_GHOST_CHECKPOINT_IDS);
 		this.userClassLoader = userClassLoader;
+		this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
+		checkpointIDCounter.start();
 
-		timer = new Timer("Checkpoint Timer", true);
+		this.timer = new Timer("Checkpoint Timer", true);
 
-		// Add shutdown hook to clean up state handles
-		shutdownHook = new Thread(new Runnable() {
-			@Override
-			public void run() {
-				try {
-					CheckpointCoordinator.this.shutdown();
-				}
-				catch (Throwable t) {
-					LOG.error("Error during shutdown of blob service via JVM shutdown hook: " +
-							t.getMessage(), t);
+		if (recoveryMode == RecoveryMode.STANDALONE) {
+			// Add shutdown hook to clean up state handles when no checkpoint recovery is
+			// possible. In case of another configured recovery mode, the checkpoints need to be
+			// available for the standby job managers.
+			this.shutdownHook = new Thread(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						CheckpointCoordinator.this.shutdown();
+					}
+					catch (Throwable t) {
+						LOG.error("Error during shutdown of checkpoint coordniator via " +
+								"JVM shutdown hook: " + t.getMessage(), t);
+					}
 				}
-			}
-		});
+			});
 
-		try {
-			// Add JVM shutdown hook to call shutdown of service
-			Runtime.getRuntime().addShutdownHook(shutdownHook);
-		}
-		catch (IllegalStateException ignored) {
-			// JVM is already shutting down. No need to do anything.
+			try {
+				// Add JVM shutdown hook to call shutdown of service
+				Runtime.getRuntime().addShutdownHook(shutdownHook);
+			}
+			catch (IllegalStateException ignored) {
+				// JVM is already shutting down. No need to do anything.
+			}
+			catch (Throwable t) {
+				LOG.error("Cannot register checkpoint coordinator shutdown hook.", t);
+			}
 		}
-		catch (Throwable t) {
-			LOG.error("Cannot register checkpoint coordinator shutdown hook.", t);
+		else {
+			this.shutdownHook = null;
 		}
 	}
 
@@ -178,41 +190,39 @@ public class CheckpointCoordinator {
 	 * After this method has been called, the coordinator does not accept and further
 	 * messages and cannot trigger any further checkpoints.
 	 */
-	public void shutdown() {
+	public void shutdown() throws Exception {
 		synchronized (lock) {
-			try {	
-				if (shutdown) {
-					return;
-				}
-				shutdown = true;
-				LOG.info("Stopping checkpoint coordinator for job " + job);
-			
-				// shut down the thread that handles the timeouts
-				timer.cancel();
-			
-				// make sure that the actor does not linger
-				if (jobStatusListener != null) {
-					jobStatusListener.tell(PoisonPill.getInstance());
-					jobStatusListener = null;
-				}
-			
-				// the scheduling thread needs also to go away
-				if (periodicScheduler != null) {
-					periodicScheduler.cancel();
-					periodicScheduler = null;
-				}
-			
-				// clear and discard all pending checkpoints
-				for (PendingCheckpoint pending : pendingCheckpoints.values()) {
-						pending.discard(userClassLoader, true);
-				}
-				pendingCheckpoints.clear();
-			
-				// clean and discard all successful checkpoints
-				for (SuccessfulCheckpoint checkpoint : completedCheckpoints) {
-					checkpoint.discard(userClassLoader);
+			try {
+				if (!shutdown) {
+					shutdown = true;
+					LOG.info("Stopping checkpoint coordinator for job " + job);
+
+					// shut down the thread that handles the timeouts
+					timer.cancel();
+
+					// make sure that the actor does not linger
+					if (jobStatusListener != null) {
+						jobStatusListener.tell(PoisonPill.getInstance());
+						jobStatusListener = null;
+					}
+
+					// the scheduling thread needs also to go away
+					if (periodicScheduler != null) {
+						periodicScheduler.cancel();
+						periodicScheduler = null;
+					}
+
+					checkpointIdCounter.stop();
+
+					// clear and discard all pending checkpoints
+					for (PendingCheckpoint pending : pendingCheckpoints.values()) {
+							pending.discard(userClassLoader, true);
+					}
+					pendingCheckpoints.clear();
+
+					// clean and discard all successful checkpoints
+					completedCheckpointStore.discardAllCheckpoints();
 				}
-				completedCheckpoints.clear();
 			}
 			finally {
 				// Remove shutdown hook to prevent resource leaks, unless this is invoked by the
@@ -244,7 +254,7 @@ public class CheckpointCoordinator {
 	 * Triggers a new checkpoint and uses the current system time as the
 	 * checkpoint time.
 	 */
-	public void triggerCheckpoint() {
+	public void triggerCheckpoint() throws Exception {
 		triggerCheckpoint(System.currentTimeMillis());
 	}
 
@@ -254,7 +264,7 @@ public class CheckpointCoordinator {
 	 * 
 	 * @param timestamp The timestamp for the checkpoint.
 	 */
-	public boolean triggerCheckpoint(final long timestamp) {
+	public boolean triggerCheckpoint(final long timestamp) throws Exception {
 		if (shutdown) {
 			LOG.error("Cannot trigger checkpoint, checkpoint coordinator has been shutdown.");
 			return false;
@@ -354,7 +364,7 @@ public class CheckpointCoordinator {
 		}
 	}
 	
-	public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) {
+	public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws Exception {
 		if (shutdown || message == null) {
 			return;
 		}
@@ -365,7 +375,7 @@ public class CheckpointCoordinator {
 		
 		final long checkpointId = message.getCheckpointId();
 
-		SuccessfulCheckpoint completed = null;
+		CompletedCheckpoint completed = null;
 		PendingCheckpoint checkpoint;
 		synchronized (lock) {
 			// we need to check inside the lock for being shutdown as well, otherwise we
@@ -380,13 +390,13 @@ public class CheckpointCoordinator {
 				if (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getState())) {
 					
 					if (checkpoint.isFullyAcknowledged()) {
+						completed = checkpoint.toCompletedCheckpoint();
+
+						completedCheckpointStore.addCheckpoint(completed);
+
 						LOG.info("Completed checkpoint " + checkpointId);
+						LOG.debug(completed.getStates().toString());
 
-						completed = checkpoint.toCompletedCheckpoint();
-						completedCheckpoints.addLast(completed);
-						if (completedCheckpoints.size() > numSuccessfulCheckpointsToRetain) {
-							completedCheckpoints.removeFirst().discard(userClassLoader);
-						}
 						pendingCheckpoints.remove(checkpointId);
 						rememberRecentCheckpointId(checkpointId);
 						
@@ -456,25 +466,30 @@ public class CheckpointCoordinator {
 	//  Checkpoint State Restoring
 	// --------------------------------------------------------------------------------------------
 
-	public void restoreLatestCheckpointedState(Map<JobVertexID, ExecutionJobVertex> tasks,
-												boolean errorIfNoCheckpoint,
-												boolean allOrNothingState) throws Exception {
+	public void restoreLatestCheckpointedState(
+			Map<JobVertexID, ExecutionJobVertex> tasks,
+			boolean errorIfNoCheckpoint,
+			boolean allOrNothingState) throws Exception {
+
 		synchronized (lock) {
 			if (shutdown) {
 				throw new IllegalStateException("CheckpointCoordinator is shut down");
 			}
-			
-			if (completedCheckpoints.isEmpty()) {
+
+			// Recover the checkpoints
+			completedCheckpointStore.recover();
+
+			// restore from the latest checkpoint
+			CompletedCheckpoint latest = completedCheckpointStore.getLatestCheckpoint();
+
+			if (latest == null) {
 				if (errorIfNoCheckpoint) {
 					throw new IllegalStateException("No completed checkpoint available");
 				} else {
 					return;
 				}
 			}
-			
-			// restore from the latest checkpoint
-			SuccessfulCheckpoint latest = completedCheckpoints.getLast();
-						
+
 			if (allOrNothingState) {
 				Map<ExecutionJobVertex, Integer> stateCounts = new HashMap<ExecutionJobVertex, Integer>();
 
@@ -519,7 +534,9 @@ public class CheckpointCoordinator {
 	}
 
 	public int getNumberOfRetainedSuccessfulCheckpoints() {
-		return this.completedCheckpoints.size();
+		synchronized (lock) {
+			return completedCheckpointStore.getNumberOfRetainedCheckpoints();
+		}
 	}
 
 	public Map<Long, PendingCheckpoint> getPendingCheckpoints() {
@@ -528,9 +545,9 @@ public class CheckpointCoordinator {
 		}
 	}
 	
-	public List<SuccessfulCheckpoint> getSuccessfulCheckpoints() {
+	public List<CompletedCheckpoint> getSuccessfulCheckpoints() throws Exception {
 		synchronized (lock) {
-			return new ArrayList<SuccessfulCheckpoint>(this.completedCheckpoints);
+			return completedCheckpointStore.getAllCheckpoints();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java
new file mode 100644
index 0000000..34b7946
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java
@@ -0,0 +1,43 @@
+/*
+ * 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.checkpoint;
+
+/**
+ * A checkpoint ID counter.
+ */
+public interface CheckpointIDCounter {
+
+	/**
+	 * Starts the {@link CheckpointIDCounter} service.
+	 */
+	void start() throws Exception;
+
+	/**
+	 * Stops the {@link CheckpointIDCounter} service.
+	 */
+	void stop() throws Exception;
+
+	/**
+	 * Atomically increments the current checkpoint ID.
+	 *
+	 * @return The previous checkpoint ID
+	 */
+	long getAndIncrement() throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java
new file mode 100644
index 0000000..aa6e94b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+
+/**
+ * A factory for per Job checkpoint recovery components.
+ */
+public interface CheckpointRecoveryFactory {
+
+	/**
+	 * The number of {@link CompletedCheckpoint} instances to retain.
+	 */
+	int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1;
+
+	/**
+	 * Starts the {@link CheckpointRecoveryFactory} service.
+	 */
+	void start();
+
+	/**
+	 * Stops the {@link CheckpointRecoveryFactory} service.
+	 */
+	void stop();
+
+	/**
+	 * Creates a {@link CompletedCheckpointStore} instance for a job.
+	 *
+	 * @param jobId           Job ID to recover checkpoints for
+	 * @param userClassLoader User code class loader of the job
+	 * @return {@link CompletedCheckpointStore} instance for the job
+	 */
+	CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader)
+			throws Exception;
+
+	/**
+	 * Creates a {@link CheckpointIDCounter} instance for a job.
+	 *
+	 * @param jobId Job ID to recover checkpoints for
+	 * @return {@link CheckpointIDCounter} instance for the job
+	 */
+	CheckpointIDCounter createCheckpointIDCounter(JobID jobId) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
new file mode 100644
index 0000000..ea3c26d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A successful checkpoint describes a checkpoint after all required tasks acknowledged it (with their state)
+ * and that is considered completed.
+ */
+public class CompletedCheckpoint implements Serializable {
+
+	private static final long serialVersionUID = -8360248179615702014L;
+
+	private final JobID job;
+	
+	private final long checkpointID;
+	
+	private final long timestamp;
+	
+	private final ArrayList<StateForTask> states;
+
+	public CompletedCheckpoint(JobID job, long checkpointID, long timestamp, ArrayList<StateForTask> states) {
+		this.job = job;
+		this.checkpointID = checkpointID;
+		this.timestamp = timestamp;
+		this.states = states;
+	}
+
+	public JobID getJobId() {
+		return job;
+	}
+
+	public long getCheckpointID() {
+		return checkpointID;
+	}
+
+	public long getTimestamp() {
+		return timestamp;
+	}
+
+	public List<StateForTask> getStates() {
+		return states;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	public void discard(ClassLoader userClassLoader) {
+		for(StateForTask state: states){
+			state.discard(userClassLoader);
+		}
+		states.clear();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return String.format("Checkpoint %d @ %d for %s", checkpointID, timestamp, job);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
new file mode 100644
index 0000000..d024aea
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java
@@ -0,0 +1,69 @@
+/*
+ * 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.checkpoint;
+
+import java.util.List;
+
+/**
+ * A bounded LIFO-queue of {@link CompletedCheckpoint} instances.
+ */
+public interface CompletedCheckpointStore {
+
+	/**
+	 * Recover available {@link CompletedCheckpoint} instances.
+	 *
+	 * <p>After a call to this method, {@link #getLatestCheckpoint()} returns the latest
+	 * available checkpoint.
+	 */
+	void recover() throws Exception;
+
+	/**
+	 * Adds a {@link CompletedCheckpoint} instance to the list of completed checkpoints.
+	 *
+	 * <p>Only a bounded number of checkpoints is kept. When exceeding the maximum number of
+	 * retained checkpoints, the oldest one will be discarded via {@link
+	 * CompletedCheckpoint#discard(ClassLoader)}.
+	 */
+	void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception;
+
+	/**
+	 * Returns the latest {@link CompletedCheckpoint} instance or <code>null</code> if none was
+	 * added.
+	 */
+	CompletedCheckpoint getLatestCheckpoint() throws Exception;
+
+	/**
+	 * Discards all added {@link CompletedCheckpoint} instances via {@link
+	 * CompletedCheckpoint#discard(ClassLoader)}.
+	 */
+	void discardAllCheckpoints() throws Exception;
+
+	/**
+	 * Returns all {@link CompletedCheckpoint} instances.
+	 *
+	 * <p>Returns an empty list if no checkpoint has been added yet.
+	 */
+	List<CompletedCheckpoint> getAllCheckpoints() throws Exception;
+
+	/**
+	 * Returns the current number of retained checkpoints.
+	 */
+	int getNumberOfRetainedCheckpoints();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index 370ae50..81159f6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -31,7 +31,7 @@ import org.apache.flink.util.SerializedValue;
 /**
  * A pending checkpoint is a checkpoint that has been started, but has not been
  * acknowledged by all tasks that need to acknowledge it. Once all tasks have
- * acknowledged it, it becomes a {@link SuccessfulCheckpoint}.
+ * acknowledged it, it becomes a {@link CompletedCheckpoint}.
  * 
  * <p>Note that the pending checkpoint, as well as the successful checkpoint keep the
  * state handles always as serialized values, never as actual values.</p>
@@ -109,13 +109,13 @@ public class PendingCheckpoint {
 		return collectedStates;
 	}
 	
-	public SuccessfulCheckpoint toCompletedCheckpoint() {
+	public CompletedCheckpoint toCompletedCheckpoint() {
 		synchronized (lock) {
 			if (discarded) {
 				throw new IllegalStateException("pending checkpoint is discarded");
 			}
 			if (notYetAcknowledgedTasks.isEmpty()) {
-				SuccessfulCheckpoint completed =  new SuccessfulCheckpoint(jobId, checkpointId,
+				CompletedCheckpoint completed =  new CompletedCheckpoint(jobId, checkpointId,
 						checkpointTimestamp, new ArrayList<StateForTask>(collectedStates));
 				discard(null, false);
 				

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java
new file mode 100644
index 0000000..052d743
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * {@link CheckpointIDCounter} instances for JobManagers running in {@link RecoveryMode#STANDALONE}.
+ *
+ * <p>Simple wrapper of an {@link AtomicLong}. This is sufficient, because job managers are not
+ * recoverable in this recovery mode.
+ */
+public class StandaloneCheckpointIDCounter implements CheckpointIDCounter {
+
+	private final AtomicLong checkpointIdCounter = new AtomicLong(1);
+
+	@Override
+	public void start() throws Exception {
+	}
+
+	@Override
+	public void stop() throws Exception {
+	}
+
+	@Override
+	public long getAndIncrement() throws Exception {
+		return checkpointIdCounter.getAndIncrement();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
new file mode 100644
index 0000000..324a0be
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+
+/**
+ * {@link CheckpointCoordinator} components in {@link RecoveryMode#STANDALONE}.
+ */
+public class StandaloneCheckpointRecoveryFactory implements CheckpointRecoveryFactory {
+
+	@Override
+	public void start() {
+		// Nothing to do
+	}
+
+	@Override
+	public void stop() {
+		// Nothing to do
+	}
+
+	@Override
+	public CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader)
+			throws Exception {
+
+		return new StandaloneCompletedCheckpointStore(CheckpointRecoveryFactory
+				.NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN, userClassLoader);
+	}
+
+	@Override
+	public CheckpointIDCounter createCheckpointIDCounter(JobID ignored) {
+		return new StandaloneCheckpointIDCounter();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
new file mode 100644
index 0000000..c31606a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#STANDALONE}.
+ */
+class StandaloneCompletedCheckpointStore implements CompletedCheckpointStore {
+
+	/** The maximum number of checkpoints to retain (at least 1). */
+	private final int maxNumberOfCheckpointsToRetain;
+
+	/** User class loader for discarding {@link CompletedCheckpoint} instances. */
+	private final ClassLoader userClassLoader;
+
+	/** The completed checkpoints. */
+	private final ArrayDeque<CompletedCheckpoint> checkpoints;
+
+	/**
+	 * Creates {@link StandaloneCompletedCheckpointStore}.
+	 *
+	 * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at
+	 *                                       least 1). Adding more checkpoints than this results
+	 *                                       in older checkpoints being discarded.
+	 * @param userClassLoader                The user class loader used to discard checkpoints
+	 */
+	public StandaloneCompletedCheckpointStore(
+			int maxNumberOfCheckpointsToRetain,
+			ClassLoader userClassLoader) {
+
+		checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint.");
+
+		this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain;
+		this.userClassLoader = checkNotNull(userClassLoader, "User class loader");
+
+		this.checkpoints = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1);
+	}
+
+	@Override
+	public void recover() throws Exception {
+		// Nothing to do
+	}
+
+	@Override
+	public void addCheckpoint(CompletedCheckpoint checkpoint) {
+		checkpoints.addLast(checkpoint);
+		if (checkpoints.size() > maxNumberOfCheckpointsToRetain) {
+			checkpoints.removeFirst().discard(userClassLoader);
+		}
+	}
+
+	@Override
+	public CompletedCheckpoint getLatestCheckpoint() {
+		return checkpoints.isEmpty() ? null : checkpoints.getLast();
+	}
+
+	@Override
+	public List<CompletedCheckpoint> getAllCheckpoints() {
+		return new ArrayList<>(checkpoints);
+	}
+
+	@Override
+	public int getNumberOfRetainedCheckpoints() {
+		return checkpoints.size();
+	}
+
+	@Override
+	public void discardAllCheckpoints() {
+		for (CompletedCheckpoint checkpoint : checkpoints) {
+			checkpoint.discard(userClassLoader);
+		}
+
+		checkpoints.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java
index 120c503..d1428f4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java
@@ -24,6 +24,11 @@ import org.apache.flink.util.SerializedValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Serializable;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Simple bean to describe the state belonging to a parallel operator.
  * Since we hold the state across execution attempts, we identify a task by its
@@ -34,8 +39,10 @@ import org.slf4j.LoggerFactory;
  * Furthermore, the state may involve user-defined classes that are not accessible without
  * the respective classloader.
  */
-public class StateForTask {
-	
+public class StateForTask implements Serializable {
+
+	private static final long serialVersionUID = -2394696997971923995L;
+
 	private static final Logger LOG = LoggerFactory.getLogger(StateForTask.class);
 
 	/** The state of the parallel operator */
@@ -48,12 +55,10 @@ public class StateForTask {
 	private final int subtask;
 	
 	public StateForTask(SerializedValue<StateHandle<?>> state, JobVertexID operatorId, int subtask) {
-	if (state == null || operatorId == null || subtask < 0) {
-			throw new IllegalArgumentException();
-		}
-		
-		this.state = state;
-		this.operatorId = operatorId;
+		this.state = checkNotNull(state, "State");
+		this.operatorId = checkNotNull(operatorId, "Operator ID");
+
+		checkArgument(subtask >= 0, "Negative subtask index");
 		this.subtask = subtask;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java
deleted file mode 100644
index be0b301..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.checkpoint;
-
-import org.apache.flink.api.common.JobID;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-
-/**
- * A successful checkpoint describes a checkpoint after all required tasks acknowledged it (with their state)
- * and that is considered completed.
- */
-public class SuccessfulCheckpoint {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(SuccessfulCheckpoint.class);
-	
-	private final JobID job;
-	
-	private final long checkpointID;
-	
-	private final long timestamp;
-	
-	private final List<StateForTask> states;
-
-
-	public SuccessfulCheckpoint(JobID job, long checkpointID, long timestamp, List<StateForTask> states) {
-		this.job = job;
-		this.checkpointID = checkpointID;
-		this.timestamp = timestamp;
-		this.states = states;
-	}
-
-	public JobID getJobId() {
-		return job;
-	}
-
-	public long getCheckpointID() {
-		return checkpointID;
-	}
-
-	public long getTimestamp() {
-		return timestamp;
-	}
-
-	public List<StateForTask> getStates() {
-		return states;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	public void discard(ClassLoader userClassLoader) {
-		for(StateForTask state: states){
-			state.discard(userClassLoader);
-		}
-		states.clear();
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return String.format("Checkpoint %d @ %d for %s", checkpointID, timestamp, job);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java
new file mode 100644
index 0000000..6673050
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.shared.SharedCount;
+import org.apache.curator.framework.recipes.shared.VersionedValue;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * {@link CheckpointIDCounter} instances for JobManagers running in {@link RecoveryMode#ZOOKEEPER}.
+ *
+ * <p>Each counter creates a ZNode:
+ * <pre>
+ * +----O /flink/checkpoint-counter/&lt;job-id&gt; 1 [persistent]
+ * .
+ * .
+ * .
+ * +----O /flink/checkpoint-counter/&lt;job-id&gt; N [persistent]
+ * </pre>
+ *
+ * <p>The checkpoints IDs are required to be ascending (per job). In order to guarantee this in case
+ * of job manager failures we use ZooKeeper to have a shared counter across job manager instances.
+ */
+public class ZooKeeperCheckpointIDCounter implements CheckpointIDCounter {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperCheckpointIDCounter.class);
+
+	/** Curator ZooKeeper client */
+	private final CuratorFramework client;
+
+	/** Path of the shared count */
+	private final String counterPath;
+
+	/** Curator recipe for shared counts */
+	private final SharedCount sharedCount;
+
+	/** Connection state listener to monitor the client connection */
+	private final SharedCountConnectionStateListener connStateListener =
+			new SharedCountConnectionStateListener();
+
+	/**
+	 * Creates a {@link ZooKeeperCheckpointIDCounter} instance.
+	 *
+	 * @param client      Curator ZooKeeper client
+	 * @param counterPath ZooKeeper path for the counter. It's sufficient to have a path per-job.
+	 * @throws Exception
+	 */
+	public ZooKeeperCheckpointIDCounter(CuratorFramework client, String counterPath) throws Exception {
+		this.client = checkNotNull(client, "Curator client");
+		this.counterPath = checkNotNull(counterPath, "Counter path");
+		this.sharedCount = new SharedCount(client, counterPath, 1);
+	}
+
+	@Override
+	public void start() throws Exception {
+		sharedCount.start();
+		client.getConnectionStateListenable().addListener(connStateListener);
+	}
+
+	@Override
+	public void stop() throws Exception {
+		sharedCount.close();
+		client.getConnectionStateListenable().removeListener(connStateListener);
+
+		LOG.info("Removing {} from ZooKeeper", counterPath);
+		client.delete().deletingChildrenIfNeeded().inBackground().forPath(counterPath);
+	}
+
+	@Override
+	public long getAndIncrement() throws Exception {
+		while (true) {
+			ConnectionState connState = connStateListener.getLastState();
+
+			if (connState != null) {
+				throw new IllegalStateException("Connection state: " + connState);
+			}
+
+			VersionedValue<Integer> current = sharedCount.getVersionedValue();
+
+			Integer newCount = current.getValue() + 1;
+
+			if (sharedCount.trySetCount(current, newCount)) {
+				return current.getValue();
+			}
+		}
+	}
+
+	/**
+	 * Connection state listener. In case of {@link ConnectionState#SUSPENDED} or {@link
+	 * ConnectionState#LOST} we are not guaranteed to read a current count from ZooKeeper.
+	 */
+	private class SharedCountConnectionStateListener implements ConnectionStateListener {
+
+		private volatile ConnectionState lastState;
+
+		@Override
+		public void stateChanged(CuratorFramework client, ConnectionState newState) {
+			if (newState == ConnectionState.SUSPENDED || newState == ConnectionState.LOST) {
+				lastState = newState;
+			}
+		}
+
+		private ConnectionState getLastState() {
+			return lastState;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
new file mode 100644
index 0000000..2659e7e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * {@link CheckpointCoordinator} components in {@link RecoveryMode#ZOOKEEPER}.
+ */
+public class ZooKeeperCheckpointRecoveryFactory implements CheckpointRecoveryFactory {
+
+	private final CuratorFramework client;
+
+	private final Configuration config;
+
+	public ZooKeeperCheckpointRecoveryFactory(CuratorFramework client, Configuration config) {
+		this.client = checkNotNull(client, "Curator client");
+		this.config = checkNotNull(config, "Configuration");
+	}
+
+	@Override
+	public void start() {
+		// Nothing to do
+	}
+
+	@Override
+	public void stop() {
+		client.close();
+	}
+
+	@Override
+	public CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader)
+			throws Exception {
+
+		return ZooKeeperUtils.createCompletedCheckpoints(client, config, jobId,
+				NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN, userClassLoader);
+	}
+
+	@Override
+	public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception {
+		return ZooKeeperUtils.createCheckpointIDCounter(client, config, jobID);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
new file mode 100644
index 0000000..62ab440
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
@@ -0,0 +1,293 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.ConcurrentModificationException;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#ZOOKEEPER}.
+ *
+ * <p>Checkpoints are added under a ZNode per job:
+ * <pre>
+ * +----O /flink/checkpoints/&lt;job-id&gt;  [persistent]
+ * .    |
+ * .    +----O /flink/checkpoints/&lt;job-id&gt;/1 [persistent]
+ * .    .                                  .
+ * .    .                                  .
+ * .    .                                  .
+ * .    +----O /flink/checkpoints/&lt;job-id&gt;/N [persistent]
+ * </pre>
+ *
+ * <p>During recovery, the latest checkpoint is read from ZooKeeper. If there is more than one,
+ * only the latest one is used and older ones are discarded (even if the maximum number
+ * of retained checkpoints is greater than one).
+ *
+ * <p>If there is a network partition and multiple JobManagers run concurrent checkpoints for the
+ * same program, it is OK to take any valid successful checkpoint as long as the "history" of
+ * checkpoints is consistent. Currently, after recovery we start out with only a single
+ * checkpoint to circumvent those situations.
+ */
+public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointStore {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperCompletedCheckpointStore.class);
+
+	/** Curator ZooKeeper client */
+	private final CuratorFramework client;
+
+	/** Completed checkpoints in ZooKeeper */
+	private final ZooKeeperStateHandleStore<CompletedCheckpoint> checkpointsInZooKeeper;
+
+	/** The maximum number of checkpoints to retain (at least 1). */
+	private final int maxNumberOfCheckpointsToRetain;
+
+	/** User class loader for discarding {@link CompletedCheckpoint} instances. */
+	private final ClassLoader userClassLoader;
+
+	/** Local completed checkpoints. */
+	private final ArrayDeque<Tuple2<StateHandle<CompletedCheckpoint>, String>> checkpointStateHandles;
+
+	/**
+	 * Creates a {@link ZooKeeperCompletedCheckpointStore} instance.
+	 *
+	 * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at
+	 *                                       least 1). Adding more checkpoints than this results
+	 *                                       in older checkpoints being discarded. On recovery,
+	 *                                       we will only start with a single checkpoint.
+	 * @param userClassLoader                The user class loader used to discard checkpoints
+	 * @param client                         The Curator ZooKeeper client
+	 * @param checkpointsPath                The ZooKeeper path for the checkpoints (needs to
+	 *                                       start with a '/')
+	 * @param stateHandleProvider            The state handle provider for checkpoints
+	 * @throws Exception
+	 */
+	public ZooKeeperCompletedCheckpointStore(
+			int maxNumberOfCheckpointsToRetain,
+			ClassLoader userClassLoader,
+			CuratorFramework client,
+			String checkpointsPath,
+			StateHandleProvider<CompletedCheckpoint> stateHandleProvider) throws Exception {
+
+		checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint.");
+
+		this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain;
+		this.userClassLoader = checkNotNull(userClassLoader, "User class loader");
+
+		checkNotNull(client, "Curator client");
+		checkNotNull(checkpointsPath, "Checkpoints path");
+		checkNotNull(stateHandleProvider, "State handle provider");
+
+		// Ensure that the checkpoints path exists
+		client.newNamespaceAwareEnsurePath(checkpointsPath)
+				.ensure(client.getZookeeperClient());
+
+		// All operations will have the path as root
+		this.client = client.usingNamespace(client.getNamespace() + checkpointsPath);
+
+		this.checkpointsInZooKeeper = new ZooKeeperStateHandleStore<>(
+				this.client, stateHandleProvider);
+
+		this.checkpointStateHandles = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1);
+
+		LOG.info("Initialized in '{}'.", checkpointsPath);
+	}
+
+	/**
+	 * Gets the latest checkpoint from ZooKeeper and removes all others.
+	 *
+	 * <p><strong>Important</strong>: Even if there are more than one checkpoint in ZooKeeper,
+	 * this will only recover the latest and discard the others. Otherwise, there is no guarantee
+	 * that the history of checkpoints is consistent.
+	 */
+	@Override
+	public void recover() throws Exception {
+		LOG.info("Recovering checkpoints from ZooKeeper.");
+
+		// Get all there is first
+		List<Tuple2<StateHandle<CompletedCheckpoint>, String>> initialCheckpoints;
+		while (true) {
+			try {
+				initialCheckpoints = checkpointsInZooKeeper.getAllSortedByName();
+				break;
+			}
+			catch (ConcurrentModificationException e) {
+				LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying.");
+			}
+		}
+
+		int numberOfInitialCheckpoints = initialCheckpoints.size();
+
+		LOG.info("Found {} checkpoints in ZooKeeper.", numberOfInitialCheckpoints);
+
+		if (numberOfInitialCheckpoints > 0) {
+			// Take the last one. This is the latest checkpoints, because path names are strictly
+			// increasing (checkpoint ID).
+			Tuple2<StateHandle<CompletedCheckpoint>, String> latest = initialCheckpoints
+					.get(numberOfInitialCheckpoints - 1);
+
+			CompletedCheckpoint latestCheckpoint = latest.f0.getState(userClassLoader);
+
+			checkpointStateHandles.add(latest);
+
+			LOG.info("Initialized with {}. Removing all older checkpoints.", latestCheckpoint);
+
+			for (int i = 0; i < numberOfInitialCheckpoints - 1; i++) {
+				try {
+					removeFromZooKeeperAndDiscardCheckpoint(initialCheckpoints.get(i));
+				}
+				catch (Exception e) {
+					LOG.error("Failed to discard checkpoint", e);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Synchronously writes the new checkpoints to ZooKeeper and asynchronously removes older ones.
+	 *
+	 * @param checkpoint Completed checkpoint to add.
+	 */
+	@Override
+	public void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception {
+		checkNotNull(checkpoint, "Checkpoint");
+
+		// First add the new one. If it fails, we don't want to loose existing data.
+		String path = String.format("/%s", checkpoint.getCheckpointID());
+
+		final StateHandle<CompletedCheckpoint> stateHandle = checkpointsInZooKeeper.add(path, checkpoint);
+
+		checkpointStateHandles.addLast(new Tuple2<>(stateHandle, path));
+
+		// Everything worked, let's remove a previous checkpoint if necessary.
+		if (checkpointStateHandles.size() > maxNumberOfCheckpointsToRetain) {
+			removeFromZooKeeperAndDiscardCheckpoint(checkpointStateHandles.removeFirst());
+		}
+
+		LOG.debug("Added {} to {}.", checkpoint, path);
+	}
+
+	@Override
+	public CompletedCheckpoint getLatestCheckpoint() throws Exception {
+		if (checkpointStateHandles.isEmpty()) {
+			return null;
+		}
+		else {
+			return checkpointStateHandles.getLast().f0.getState(userClassLoader);
+		}
+	}
+
+	@Override
+	public List<CompletedCheckpoint> getAllCheckpoints() throws Exception {
+		List<CompletedCheckpoint> checkpoints = new ArrayList<>(checkpointStateHandles.size());
+
+		for (Tuple2<StateHandle<CompletedCheckpoint>, String> stateHandle : checkpointStateHandles) {
+			checkpoints.add(stateHandle.f0.getState(userClassLoader));
+		}
+
+		return checkpoints;
+	}
+
+	@Override
+	public int getNumberOfRetainedCheckpoints() {
+		return checkpointStateHandles.size();
+	}
+
+	@Override
+	public void discardAllCheckpoints() throws Exception {
+		for (Tuple2<StateHandle<CompletedCheckpoint>, String> checkpoint : checkpointStateHandles) {
+			try {
+				removeFromZooKeeperAndDiscardCheckpoint(checkpoint);
+			}
+			catch (Exception e) {
+				LOG.error("Failed to discard checkpoint.", e);
+			}
+		}
+
+		checkpointStateHandles.clear();
+
+		String path = "/" + client.getNamespace();
+
+		LOG.info("Removing {} from ZooKeeper", path);
+		ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, true);
+	}
+
+	/**
+	 * Removes the state handle from ZooKeeper, discards the checkpoints, and the state handle.
+	 */
+	private void removeFromZooKeeperAndDiscardCheckpoint(
+			final Tuple2<StateHandle<CompletedCheckpoint>, String> stateHandleAndPath) throws Exception {
+
+		final BackgroundCallback callback = new BackgroundCallback() {
+			@Override
+			public void processResult(CuratorFramework client, CuratorEvent event) throws Exception {
+				try {
+					if (event.getType() == CuratorEventType.DELETE) {
+						if (event.getResultCode() == 0) {
+							// The checkpoint
+							CompletedCheckpoint checkpoint = stateHandleAndPath
+									.f0.getState(userClassLoader);
+
+							checkpoint.discard(userClassLoader);
+
+							// Discard the state handle
+							stateHandleAndPath.f0.discardState();
+
+							// Discard the checkpoint
+							LOG.debug("Discarded " + checkpoint);
+						}
+						else {
+							throw new IllegalStateException("Unexpected result code " +
+									event.getResultCode() + " in '" + event + "' callback.");
+						}
+					}
+					else {
+						throw new IllegalStateException("Unexpected event type " +
+								event.getType() + " in '" + event + "' callback.");
+					}
+				}
+				catch (Exception e) {
+					LOG.error("Failed to discard checkpoint.", e);
+				}
+			}
+		};
+
+		// Remove state handle from ZooKeeper first. If this fails, we can still recover, but if
+		// we remove a state handle and fail to remove it from ZooKeeper, we end up in an
+		// inconsistent state.
+		checkpointsInZooKeeper.remove(stateHandleAndPath.f1, callback);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/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 ef00484..9430d80 100755
--- 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
@@ -30,6 +30,8 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
@@ -39,6 +41,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
@@ -110,8 +113,6 @@ public class ExecutionGraph implements Serializable {
 	/** The log object used for debugging. */
 	static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class);
 	
-	private static final int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1;
-
 	// --------------------------------------------------------------------------------------------
 
 	/** The lock used to secure all access to mutable fields, especially the tracking of progress
@@ -347,7 +348,11 @@ public class ExecutionGraph implements Serializable {
 			List<ExecutionJobVertex> verticesToWaitFor,
 			List<ExecutionJobVertex> verticesToCommitTo,
 			ActorSystem actorSystem,
-			UUID leaderSessionID) {
+			UUID leaderSessionID,
+			CheckpointIDCounter checkpointIDCounter,
+			CompletedCheckpointStore completedCheckpointStore,
+			RecoveryMode recoveryMode) throws Exception {
+
 		// simple sanity checks
 		if (interval < 10 || checkpointTimeout < 10) {
 			throw new IllegalArgumentException();
@@ -367,12 +372,14 @@ public class ExecutionGraph implements Serializable {
 		snapshotCheckpointsEnabled = true;
 		checkpointCoordinator = new CheckpointCoordinator(
 				jobID,
-				NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN,
 				checkpointTimeout,
 				tasksToTrigger,
 				tasksToWaitFor,
 				tasksToCommitTo,
-				userClassLoader);
+				userClassLoader,
+				checkpointIDCounter,
+				completedCheckpointStore,
+				recoveryMode);
 		
 		// the periodic checkpoint scheduler is activated and deactivated as a result of
 		// job status changes (running -> on, all other states -> off)
@@ -382,8 +389,14 @@ public class ExecutionGraph implements Serializable {
 						interval,
 						leaderSessionID));
 	}
-	
-	public void disableSnaphotCheckpointing() {
+
+	/**
+	 * Disables checkpointing.
+	 *
+	 * <p>The shutdown of the checkpoint coordinator might block. Make sure that calls to this
+	 * method don't block the job manager actor and run asynchronously.
+	 */
+	public void disableSnaphotCheckpointing() throws Exception {
 		if (state != JobStatus.CREATED) {
 			throw new IllegalStateException("Job must be in CREATED state");
 		}
@@ -773,6 +786,20 @@ public class ExecutionGraph implements Serializable {
 	}
 
 	/**
+	 * Restores the latest checkpointed state.
+	 *
+	 * <p>The recovery of checkpoints might block. Make sure that calls to this method don't
+	 * block the job manager actor and run asynchronously.
+	 */
+	public void restoreLatestCheckpointedState() throws Exception {
+		synchronized (progressLock) {
+			if (checkpointCoordinator != null) {
+				checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false);
+			}
+		}
+	}
+
+	/**
 	 * This method cleans fields that are irrelevant for the archived execution attempt.
 	 */
 	public void prepareForArchiving() {
@@ -886,7 +913,13 @@ public class ExecutionGraph implements Serializable {
 									}
 								}, executionContext);
 							} else {
-								restart();
+								future(new Callable<Object>() {
+									@Override
+									public Object call() throws Exception {
+										restart();
+										return null;
+									}
+								}, executionContext);
 							}
 							break;
 						}
@@ -906,7 +939,7 @@ public class ExecutionGraph implements Serializable {
 			}
 		}
 	}
-	
+
 	private void postRunCleanup() {
 		try {
 			CheckpointCoordinator coord = this.checkpointCoordinator;

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 6b36e2d..a64d63c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -538,4 +538,9 @@ public class JobGraph implements Serializable {
 			}
 		}
 	}
+
+	@Override
+	public String toString() {
+		return "JobGraph(jobId: " + jobID + ")";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
index 2e75b19..17322d8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
@@ -35,6 +35,19 @@ public enum RecoveryMode {
 	ZOOKEEPER;
 
 	/**
+	 * Return the configured {@link RecoveryMode}.
+	 *
+	 * @param config The config to parse
+	 * @return Configured recovery mode or {@link ConfigConstants#DEFAULT_RECOVERY_MODE} if not
+	 * configured.
+	 */
+	public static RecoveryMode fromConfig(Configuration config) {
+		return RecoveryMode.valueOf(config.getString(
+				ConfigConstants.RECOVERY_MODE,
+				ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase());
+	}
+
+	/**
 	 * Returns true if the defined recovery mode supports high availability.
 	 *
 	 * @param configuration Configuration which contains the recovery mode

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
new file mode 100644
index 0000000..db36f92
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import scala.Option;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * {@link SubmittedJobGraph} instances for JobManagers running in {@link RecoveryMode#STANDALONE}.
+ *
+ * <p>All operations are NoOps, because {@link JobGraph} instances cannot be recovered in this
+ * recovery mode.
+ */
+public class StandaloneSubmittedJobGraphStore implements SubmittedJobGraphStore {
+
+	@Override
+	public void start(SubmittedJobGraphListener jobGraphListener) throws Exception {
+		// Nothing to do
+	}
+
+	@Override
+	public void stop() {
+		// Nothing to do
+	}
+
+	@Override
+	public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception {
+		// Nothing to do
+	}
+
+	@Override
+	public void removeJobGraph(JobID jobId) throws Exception {
+		// Nothing to do
+	}
+
+	@Override
+	public Option<SubmittedJobGraph> recoverJobGraph(JobID jobId) throws Exception {
+		return Option.empty();
+	}
+
+	@Override
+	public List<SubmittedJobGraph> recoverJobGraphs() throws Exception {
+		return Collections.emptyList();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java
new file mode 100644
index 0000000..48da3b8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+
+import java.io.Serializable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A recoverable {@link JobGraph} and {@link JobInfo}.
+ */
+public class SubmittedJobGraph implements Serializable {
+
+	private static final long serialVersionUID = 2836099271734771825L;
+
+	/** The submitted {@link JobGraph} */
+	private final JobGraph jobGraph;
+
+	/** The {@link JobInfo}. */
+	private final JobInfo jobInfo;
+
+	/**
+	 * Creates a {@link SubmittedJobGraph}.
+	 *
+	 * @param jobGraph The submitted {@link JobGraph}
+	 * @param jobInfo  The {@link JobInfo}
+	 */
+	public SubmittedJobGraph(JobGraph jobGraph, JobInfo jobInfo) {
+		this.jobGraph = checkNotNull(jobGraph, "Job graph");
+		this.jobInfo = checkNotNull(jobInfo, "Job info");
+	}
+
+	/**
+	 * Returns the submitted {@link JobGraph}.
+	 */
+	public JobGraph getJobGraph() {
+		return jobGraph;
+	}
+
+	/**
+	 * Returns the {@link JobID} of the submitted {@link JobGraph}.
+	 */
+	public JobID getJobId() {
+		return jobGraph.getJobID();
+	}
+
+	/**
+	 * Returns the {@link JobInfo} of the client who submitted the {@link JobGraph}.
+	 */
+	public JobInfo getJobInfo() throws Exception {
+		return jobInfo;
+	}
+
+	@Override
+	public String toString() {
+		return String.format("SubmittedJobGraph(%s, %s)", jobGraph, jobInfo);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/73c73e92/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
new file mode 100644
index 0000000..bd628cd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import scala.Option;
+
+import java.util.List;
+
+/**
+ * {@link SubmittedJobGraph} instances for recovery.
+ */
+public interface SubmittedJobGraphStore {
+
+	/**
+	 * Starts the {@link SubmittedJobGraphStore} service.
+	 */
+	void start(SubmittedJobGraphListener jobGraphListener) throws Exception;
+
+	/**
+	 * Stops the {@link SubmittedJobGraphStore} service.
+	 */
+	void stop() throws Exception;
+
+	/**
+	 * Returns a list of all submitted {@link JobGraph} instances.
+	 */
+	List<SubmittedJobGraph> recoverJobGraphs() throws Exception;
+
+	/**
+	 * Returns the {@link SubmittedJobGraph} with the given {@link JobID}.
+	 *
+	 * <p>An Exception is thrown, if no job graph with the given ID exists.
+	 */
+	Option<SubmittedJobGraph> recoverJobGraph(JobID jobId) throws Exception;
+
+	/**
+	 * Adds the {@link SubmittedJobGraph} instance.
+	 *
+	 * <p>If a job graph with the same {@link JobID} exists, it is replaced.
+	 */
+	void putJobGraph(SubmittedJobGraph jobGraph) throws Exception;
+
+	/**
+	 * Removes the {@link SubmittedJobGraph} with the given {@link JobID} if it exists.
+	 */
+	void removeJobGraph(JobID jobId) throws Exception;
+
+	/**
+	 * A listener for {@link SubmittedJobGraph} instances. This is used to react to races between
+	 * multiple running {@link SubmittedJobGraphStore} instances (on multiple job managers).
+	 */
+	interface SubmittedJobGraphListener {
+
+		/**
+		 * Callback for {@link SubmittedJobGraph} instances added by a different {@link
+		 * SubmittedJobGraphStore} instance.
+		 *
+		 * <p><strong>Important:</strong> It is possible to get false positives and be notified
+		 * about a job graph, which was added by this instance.
+		 *
+		 * @param jobId The {@link JobID} of the added job graph
+		 */
+		void onAddedJobGraph(JobID jobId);
+
+		/**
+		 * Callback for {@link SubmittedJobGraph} instances removed by a different {@link
+		 * SubmittedJobGraphStore} instance.
+		 *
+		 * @param jobId The {@link JobID} of the removed job graph
+		 */
+		void onRemovedJobGraph(JobID jobId);
+
+	}
+
+}


[12/47] flink git commit: [FLINK-2793] [runtime-web] Redirect to leader in non-standalone mode

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/77fc0cc4/flink-runtime-web/web-dashboard/web/js/vendor.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/vendor.js b/flink-runtime-web/web-dashboard/web/js/vendor.js
deleted file mode 100644
index 52bc81c..0000000
--- a/flink-runtime-web/web-dashboard/web/js/vendor.js
+++ /dev/null
@@ -1,81087 +0,0 @@
-/*!
- * jQuery JavaScript Library v2.1.4
- * http://jquery.com/
- *
- * Includes Sizzle.js
- * http://sizzlejs.com/
- *
- * Copyright 2005, 2014 jQuery Foundation, Inc. and other contributors
- * Released under the MIT license
- * http://jquery.org/license
- *
- * Date: 2015-04-28T16:01Z
- */
-
-(function( global, factory ) {
-
-	if ( typeof module === "object" && typeof module.exports === "object" ) {
-		// For CommonJS and CommonJS-like environments where a proper `window`
-		// is present, execute the factory and get jQuery.
-		// For environments that do not have a `window` with a `document`
-		// (such as Node.js), expose a factory as module.exports.
-		// This accentuates the need for the creation of a real `window`.
-		// e.g. var jQuery = require("jquery")(window);
-		// See ticket #14549 for more info.
-		module.exports = global.document ?
-			factory( global, true ) :
-			function( w ) {
-				if ( !w.document ) {
-					throw new Error( "jQuery requires a window with a document" );
-				}
-				return factory( w );
-			};
-	} else {
-		factory( global );
-	}
-
-// Pass this if window is not defined yet
-}(typeof window !== "undefined" ? window : this, function( window, noGlobal ) {
-
-// Support: Firefox 18+
-// Can't be in strict mode, several libs including ASP.NET trace
-// the stack via arguments.caller.callee and Firefox dies if
-// you try to trace through "use strict" call chains. (#13335)
-//
-
-var arr = [];
-
-var slice = arr.slice;
-
-var concat = arr.concat;
-
-var push = arr.push;
-
-var indexOf = arr.indexOf;
-
-var class2type = {};
-
-var toString = class2type.toString;
-
-var hasOwn = class2type.hasOwnProperty;
-
-var support = {};
-
-
-
-var
-	// Use the correct document accordingly with window argument (sandbox)
-	document = window.document,
-
-	version = "2.1.4",
-
-	// Define a local copy of jQuery
-	jQuery = function( selector, context ) {
-		// The jQuery object is actually just the init constructor 'enhanced'
-		// Need init if jQuery is called (just allow error to be thrown if not included)
-		return new jQuery.fn.init( selector, context );
-	},
-
-	// Support: Android<4.1
-	// Make sure we trim BOM and NBSP
-	rtrim = /^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,
-
-	// Matches dashed string for camelizing
-	rmsPrefix = /^-ms-/,
-	rdashAlpha = /-([\da-z])/gi,
-
-	// Used by jQuery.camelCase as callback to replace()
-	fcamelCase = function( all, letter ) {
-		return letter.toUpperCase();
-	};
-
-jQuery.fn = jQuery.prototype = {
-	// The current version of jQuery being used
-	jquery: version,
-
-	constructor: jQuery,
-
-	// Start with an empty selector
-	selector: "",
-
-	// The default length of a jQuery object is 0
-	length: 0,
-
-	toArray: function() {
-		return slice.call( this );
-	},
-
-	// Get the Nth element in the matched element set OR
-	// Get the whole matched element set as a clean array
-	get: function( num ) {
-		return num != null ?
-
-			// Return just the one element from the set
-			( num < 0 ? this[ num + this.length ] : this[ num ] ) :
-
-			// Return all the elements in a clean array
-			slice.call( this );
-	},
-
-	// Take an array of elements and push it onto the stack
-	// (returning the new matched element set)
-	pushStack: function( elems ) {
-
-		// Build a new jQuery matched element set
-		var ret = jQuery.merge( this.constructor(), elems );
-
-		// Add the old object onto the stack (as a reference)
-		ret.prevObject = this;
-		ret.context = this.context;
-
-		// Return the newly-formed element set
-		return ret;
-	},
-
-	// Execute a callback for every element in the matched set.
-	// (You can seed the arguments with an array of args, but this is
-	// only used internally.)
-	each: function( callback, args ) {
-		return jQuery.each( this, callback, args );
-	},
-
-	map: function( callback ) {
-		return this.pushStack( jQuery.map(this, function( elem, i ) {
-			return callback.call( elem, i, elem );
-		}));
-	},
-
-	slice: function() {
-		return this.pushStack( slice.apply( this, arguments ) );
-	},
-
-	first: function() {
-		return this.eq( 0 );
-	},
-
-	last: function() {
-		return this.eq( -1 );
-	},
-
-	eq: function( i ) {
-		var len = this.length,
-			j = +i + ( i < 0 ? len : 0 );
-		return this.pushStack( j >= 0 && j < len ? [ this[j] ] : [] );
-	},
-
-	end: function() {
-		return this.prevObject || this.constructor(null);
-	},
-
-	// For internal use only.
-	// Behaves like an Array's method, not like a jQuery method.
-	push: push,
-	sort: arr.sort,
-	splice: arr.splice
-};
-
-jQuery.extend = jQuery.fn.extend = function() {
-	var options, name, src, copy, copyIsArray, clone,
-		target = arguments[0] || {},
-		i = 1,
-		length = arguments.length,
-		deep = false;
-
-	// Handle a deep copy situation
-	if ( typeof target === "boolean" ) {
-		deep = target;
-
-		// Skip the boolean and the target
-		target = arguments[ i ] || {};
-		i++;
-	}
-
-	// Handle case when target is a string or something (possible in deep copy)
-	if ( typeof target !== "object" && !jQuery.isFunction(target) ) {
-		target = {};
-	}
-
-	// Extend jQuery itself if only one argument is passed
-	if ( i === length ) {
-		target = this;
-		i--;
-	}
-
-	for ( ; i < length; i++ ) {
-		// Only deal with non-null/undefined values
-		if ( (options = arguments[ i ]) != null ) {
-			// Extend the base object
-			for ( name in options ) {
-				src = target[ name ];
-				copy = options[ name ];
-
-				// Prevent never-ending loop
-				if ( target === copy ) {
-					continue;
-				}
-
-				// Recurse if we're merging plain objects or arrays
-				if ( deep && copy && ( jQuery.isPlainObject(copy) || (copyIsArray = jQuery.isArray(copy)) ) ) {
-					if ( copyIsArray ) {
-						copyIsArray = false;
-						clone = src && jQuery.isArray(src) ? src : [];
-
-					} else {
-						clone = src && jQuery.isPlainObject(src) ? src : {};
-					}
-
-					// Never move original objects, clone them
-					target[ name ] = jQuery.extend( deep, clone, copy );
-
-				// Don't bring in undefined values
-				} else if ( copy !== undefined ) {
-					target[ name ] = copy;
-				}
-			}
-		}
-	}
-
-	// Return the modified object
-	return target;
-};
-
-jQuery.extend({
-	// Unique for each copy of jQuery on the page
-	expando: "jQuery" + ( version + Math.random() ).replace( /\D/g, "" ),
-
-	// Assume jQuery is ready without the ready module
-	isReady: true,
-
-	error: function( msg ) {
-		throw new Error( msg );
-	},
-
-	noop: function() {},
-
-	isFunction: function( obj ) {
-		return jQuery.type(obj) === "function";
-	},
-
-	isArray: Array.isArray,
-
-	isWindow: function( obj ) {
-		return obj != null && obj === obj.window;
-	},
-
-	isNumeric: function( obj ) {
-		// parseFloat NaNs numeric-cast false positives (null|true|false|"")
-		// ...but misinterprets leading-number strings, particularly hex literals ("0x...")
-		// subtraction forces infinities to NaN
-		// adding 1 corrects loss of precision from parseFloat (#15100)
-		return !jQuery.isArray( obj ) && (obj - parseFloat( obj ) + 1) >= 0;
-	},
-
-	isPlainObject: function( obj ) {
-		// Not plain objects:
-		// - Any object or value whose internal [[Class]] property is not "[object Object]"
-		// - DOM nodes
-		// - window
-		if ( jQuery.type( obj ) !== "object" || obj.nodeType || jQuery.isWindow( obj ) ) {
-			return false;
-		}
-
-		if ( obj.constructor &&
-				!hasOwn.call( obj.constructor.prototype, "isPrototypeOf" ) ) {
-			return false;
-		}
-
-		// If the function hasn't returned already, we're confident that
-		// |obj| is a plain object, created by {} or constructed with new Object
-		return true;
-	},
-
-	isEmptyObject: function( obj ) {
-		var name;
-		for ( name in obj ) {
-			return false;
-		}
-		return true;
-	},
-
-	type: function( obj ) {
-		if ( obj == null ) {
-			return obj + "";
-		}
-		// Support: Android<4.0, iOS<6 (functionish RegExp)
-		return typeof obj === "object" || typeof obj === "function" ?
-			class2type[ toString.call(obj) ] || "object" :
-			typeof obj;
-	},
-
-	// Evaluates a script in a global context
-	globalEval: function( code ) {
-		var script,
-			indirect = eval;
-
-		code = jQuery.trim( code );
-
-		if ( code ) {
-			// If the code includes a valid, prologue position
-			// strict mode pragma, execute code by injecting a
-			// script tag into the document.
-			if ( code.indexOf("use strict") === 1 ) {
-				script = document.createElement("script");
-				script.text = code;
-				document.head.appendChild( script ).parentNode.removeChild( script );
-			} else {
-			// Otherwise, avoid the DOM node creation, insertion
-			// and removal by using an indirect global eval
-				indirect( code );
-			}
-		}
-	},
-
-	// Convert dashed to camelCase; used by the css and data modules
-	// Support: IE9-11+
-	// Microsoft forgot to hump their vendor prefix (#9572)
-	camelCase: function( string ) {
-		return string.replace( rmsPrefix, "ms-" ).replace( rdashAlpha, fcamelCase );
-	},
-
-	nodeName: function( elem, name ) {
-		return elem.nodeName && elem.nodeName.toLowerCase() === name.toLowerCase();
-	},
-
-	// args is for internal usage only
-	each: function( obj, callback, args ) {
-		var value,
-			i = 0,
-			length = obj.length,
-			isArray = isArraylike( obj );
-
-		if ( args ) {
-			if ( isArray ) {
-				for ( ; i < length; i++ ) {
-					value = callback.apply( obj[ i ], args );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			} else {
-				for ( i in obj ) {
-					value = callback.apply( obj[ i ], args );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			}
-
-		// A special, fast, case for the most common use of each
-		} else {
-			if ( isArray ) {
-				for ( ; i < length; i++ ) {
-					value = callback.call( obj[ i ], i, obj[ i ] );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			} else {
-				for ( i in obj ) {
-					value = callback.call( obj[ i ], i, obj[ i ] );
-
-					if ( value === false ) {
-						break;
-					}
-				}
-			}
-		}
-
-		return obj;
-	},
-
-	// Support: Android<4.1
-	trim: function( text ) {
-		return text == null ?
-			"" :
-			( text + "" ).replace( rtrim, "" );
-	},
-
-	// results is for internal usage only
-	makeArray: function( arr, results ) {
-		var ret = results || [];
-
-		if ( arr != null ) {
-			if ( isArraylike( Object(arr) ) ) {
-				jQuery.merge( ret,
-					typeof arr === "string" ?
-					[ arr ] : arr
-				);
-			} else {
-				push.call( ret, arr );
-			}
-		}
-
-		return ret;
-	},
-
-	inArray: function( elem, arr, i ) {
-		return arr == null ? -1 : indexOf.call( arr, elem, i );
-	},
-
-	merge: function( first, second ) {
-		var len = +second.length,
-			j = 0,
-			i = first.length;
-
-		for ( ; j < len; j++ ) {
-			first[ i++ ] = second[ j ];
-		}
-
-		first.length = i;
-
-		return first;
-	},
-
-	grep: function( elems, callback, invert ) {
-		var callbackInverse,
-			matches = [],
-			i = 0,
-			length = elems.length,
-			callbackExpect = !invert;
-
-		// Go through the array, only saving the items
-		// that pass the validator function
-		for ( ; i < length; i++ ) {
-			callbackInverse = !callback( elems[ i ], i );
-			if ( callbackInverse !== callbackExpect ) {
-				matches.push( elems[ i ] );
-			}
-		}
-
-		return matches;
-	},
-
-	// arg is for internal usage only
-	map: function( elems, callback, arg ) {
-		var value,
-			i = 0,
-			length = elems.length,
-			isArray = isArraylike( elems ),
-			ret = [];
-
-		// Go through the array, translating each of the items to their new values
-		if ( isArray ) {
-			for ( ; i < length; i++ ) {
-				value = callback( elems[ i ], i, arg );
-
-				if ( value != null ) {
-					ret.push( value );
-				}
-			}
-
-		// Go through every key on the object,
-		} else {
-			for ( i in elems ) {
-				value = callback( elems[ i ], i, arg );
-
-				if ( value != null ) {
-					ret.push( value );
-				}
-			}
-		}
-
-		// Flatten any nested arrays
-		return concat.apply( [], ret );
-	},
-
-	// A global GUID counter for objects
-	guid: 1,
-
-	// Bind a function to a context, optionally partially applying any
-	// arguments.
-	proxy: function( fn, context ) {
-		var tmp, args, proxy;
-
-		if ( typeof context === "string" ) {
-			tmp = fn[ context ];
-			context = fn;
-			fn = tmp;
-		}
-
-		// Quick check to determine if target is callable, in the spec
-		// this throws a TypeError, but we will just return undefined.
-		if ( !jQuery.isFunction( fn ) ) {
-			return undefined;
-		}
-
-		// Simulated bind
-		args = slice.call( arguments, 2 );
-		proxy = function() {
-			return fn.apply( context || this, args.concat( slice.call( arguments ) ) );
-		};
-
-		// Set the guid of unique handler to the same of original handler, so it can be removed
-		proxy.guid = fn.guid = fn.guid || jQuery.guid++;
-
-		return proxy;
-	},
-
-	now: Date.now,
-
-	// jQuery.support is not used in Core but other projects attach their
-	// properties to it so it needs to exist.
-	support: support
-});
-
-// Populate the class2type map
-jQuery.each("Boolean Number String Function Array Date RegExp Object Error".split(" "), function(i, name) {
-	class2type[ "[object " + name + "]" ] = name.toLowerCase();
-});
-
-function isArraylike( obj ) {
-
-	// Support: iOS 8.2 (not reproducible in simulator)
-	// `in` check used to prevent JIT error (gh-2145)
-	// hasOwn isn't used here due to false negatives
-	// regarding Nodelist length in IE
-	var length = "length" in obj && obj.length,
-		type = jQuery.type( obj );
-
-	if ( type === "function" || jQuery.isWindow( obj ) ) {
-		return false;
-	}
-
-	if ( obj.nodeType === 1 && length ) {
-		return true;
-	}
-
-	return type === "array" || length === 0 ||
-		typeof length === "number" && length > 0 && ( length - 1 ) in obj;
-}
-var Sizzle =
-/*!
- * Sizzle CSS Selector Engine v2.2.0-pre
- * http://sizzlejs.com/
- *
- * Copyright 2008, 2014 jQuery Foundation, Inc. and other contributors
- * Released under the MIT license
- * http://jquery.org/license
- *
- * Date: 2014-12-16
- */
-(function( window ) {
-
-var i,
-	support,
-	Expr,
-	getText,
-	isXML,
-	tokenize,
-	compile,
-	select,
-	outermostContext,
-	sortInput,
-	hasDuplicate,
-
-	// Local document vars
-	setDocument,
-	document,
-	docElem,
-	documentIsHTML,
-	rbuggyQSA,
-	rbuggyMatches,
-	matches,
-	contains,
-
-	// Instance-specific data
-	expando = "sizzle" + 1 * new Date(),
-	preferredDoc = window.document,
-	dirruns = 0,
-	done = 0,
-	classCache = createCache(),
-	tokenCache = createCache(),
-	compilerCache = createCache(),
-	sortOrder = function( a, b ) {
-		if ( a === b ) {
-			hasDuplicate = true;
-		}
-		return 0;
-	},
-
-	// General-purpose constants
-	MAX_NEGATIVE = 1 << 31,
-
-	// Instance methods
-	hasOwn = ({}).hasOwnProperty,
-	arr = [],
-	pop = arr.pop,
-	push_native = arr.push,
-	push = arr.push,
-	slice = arr.slice,
-	// Use a stripped-down indexOf as it's faster than native
-	// http://jsperf.com/thor-indexof-vs-for/5
-	indexOf = function( list, elem ) {
-		var i = 0,
-			len = list.length;
-		for ( ; i < len; i++ ) {
-			if ( list[i] === elem ) {
-				return i;
-			}
-		}
-		return -1;
-	},
-
-	booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",
-
-	// Regular expressions
-
-	// Whitespace characters http://www.w3.org/TR/css3-selectors/#whitespace
-	whitespace = "[\\x20\\t\\r\\n\\f]",
-	// http://www.w3.org/TR/css3-syntax/#characters
-	characterEncoding = "(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",
-
-	// Loosely modeled on CSS identifier characters
-	// An unquoted value should be a CSS identifier http://www.w3.org/TR/css3-selectors/#attribute-selectors
-	// Proper syntax: http://www.w3.org/TR/CSS21/syndata.html#value-def-identifier
-	identifier = characterEncoding.replace( "w", "w#" ),
-
-	// Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
-	attributes = "\\[" + whitespace + "*(" + characterEncoding + ")(?:" + whitespace +
-		// Operator (capture 2)
-		"*([*^$|!~]?=)" + whitespace +
-		// "Attribute values must be CSS identifiers [capture 5] or strings [capture 3 or capture 4]"
-		"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + whitespace +
-		"*\\]",
-
-	pseudos = ":(" + characterEncoding + ")(?:\\((" +
-		// To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
-		// 1. quoted (capture 3; capture 4 or capture 5)
-		"('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
-		// 2. simple (capture 6)
-		"((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
-		// 3. anything else (capture 2)
-		".*" +
-		")\\)|)",
-
-	// Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
-	rwhitespace = new RegExp( whitespace + "+", "g" ),
-	rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + whitespace + "+$", "g" ),
-
-	rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
-	rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + "*" ),
-
-	rattributeQuotes = new RegExp( "=" + whitespace + "*([^\\]'\"]*?)" + whitespace + "*\\]", "g" ),
-
-	rpseudo = new RegExp( pseudos ),
-	ridentifier = new RegExp( "^" + identifier + "$" ),
-
-	matchExpr = {
-		"ID": new RegExp( "^#(" + characterEncoding + ")" ),
-		"CLASS": new RegExp( "^\\.(" + characterEncoding + ")" ),
-		"TAG": new RegExp( "^(" + characterEncoding.replace( "w", "w*" ) + ")" ),
-		"ATTR": new RegExp( "^" + attributes ),
-		"PSEUDO": new RegExp( "^" + pseudos ),
-		"CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + whitespace +
-			"*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + whitespace +
-			"*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
-		"bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
-		// For use in libraries implementing .is()
-		// We use this for POS matching in `select`
-		"needsContext": new RegExp( "^" + whitespace + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" +
-			whitespace + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
-	},
-
-	rinputs = /^(?:input|select|textarea|button)$/i,
-	rheader = /^h\d$/i,
-
-	rnative = /^[^{]+\{\s*\[native \w/,
-
-	// Easily-parseable/retrievable ID or TAG or CLASS selectors
-	rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
-
-	rsibling = /[+~]/,
-	rescape = /'|\\/g,
-
-	// CSS escapes http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
-	runescape = new RegExp( "\\\\([\\da-f]{1,6}" + whitespace + "?|(" + whitespace + ")|.)", "ig" ),
-	funescape = function( _, escaped, escapedWhitespace ) {
-		var high = "0x" + escaped - 0x10000;
-		// NaN means non-codepoint
-		// Support: Firefox<24
-		// Workaround erroneous numeric interpretation of +"0x"
-		return high !== high || escapedWhitespace ?
-			escaped :
-			high < 0 ?
-				// BMP codepoint
-				String.fromCharCode( high + 0x10000 ) :
-				// Supplemental Plane codepoint (surrogate pair)
-				String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
-	},
-
-	// Used for iframes
-	// See setDocument()
-	// Removing the function wrapper causes a "Permission Denied"
-	// error in IE
-	unloadHandler = function() {
-		setDocument();
-	};
-
-// Optimize for push.apply( _, NodeList )
-try {
-	push.apply(
-		(arr = slice.call( preferredDoc.childNodes )),
-		preferredDoc.childNodes
-	);
-	// Support: Android<4.0
-	// Detect silently failing push.apply
-	arr[ preferredDoc.childNodes.length ].nodeType;
-} catch ( e ) {
-	push = { apply: arr.length ?
-
-		// Leverage slice if possible
-		function( target, els ) {
-			push_native.apply( target, slice.call(els) );
-		} :
-
-		// Support: IE<9
-		// Otherwise append directly
-		function( target, els ) {
-			var j = target.length,
-				i = 0;
-			// Can't trust NodeList.length
-			while ( (target[j++] = els[i++]) ) {}
-			target.length = j - 1;
-		}
-	};
-}
-
-function Sizzle( selector, context, results, seed ) {
-	var match, elem, m, nodeType,
-		// QSA vars
-		i, groups, old, nid, newContext, newSelector;
-
-	if ( ( context ? context.ownerDocument || context : preferredDoc ) !== document ) {
-		setDocument( context );
-	}
-
-	context = context || document;
-	results = results || [];
-	nodeType = context.nodeType;
-
-	if ( typeof selector !== "string" || !selector ||
-		nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
-
-		return results;
-	}
-
-	if ( !seed && documentIsHTML ) {
-
-		// Try to shortcut find operations when possible (e.g., not under DocumentFragment)
-		if ( nodeType !== 11 && (match = rquickExpr.exec( selector )) ) {
-			// Speed-up: Sizzle("#ID")
-			if ( (m = match[1]) ) {
-				if ( nodeType === 9 ) {
-					elem = context.getElementById( m );
-					// Check parentNode to catch when Blackberry 4.6 returns
-					// nodes that are no longer in the document (jQuery #6963)
-					if ( elem && elem.parentNode ) {
-						// Handle the case where IE, Opera, and Webkit return items
-						// by name instead of ID
-						if ( elem.id === m ) {
-							results.push( elem );
-							return results;
-						}
-					} else {
-						return results;
-					}
-				} else {
-					// Context is not a document
-					if ( context.ownerDocument && (elem = context.ownerDocument.getElementById( m )) &&
-						contains( context, elem ) && elem.id === m ) {
-						results.push( elem );
-						return results;
-					}
-				}
-
-			// Speed-up: Sizzle("TAG")
-			} else if ( match[2] ) {
-				push.apply( results, context.getElementsByTagName( selector ) );
-				return results;
-
-			// Speed-up: Sizzle(".CLASS")
-			} else if ( (m = match[3]) && support.getElementsByClassName ) {
-				push.apply( results, context.getElementsByClassName( m ) );
-				return results;
-			}
-		}
-
-		// QSA path
-		if ( support.qsa && (!rbuggyQSA || !rbuggyQSA.test( selector )) ) {
-			nid = old = expando;
-			newContext = context;
-			newSelector = nodeType !== 1 && selector;
-
-			// qSA works strangely on Element-rooted queries
-			// We can work around this by specifying an extra ID on the root
-			// and working up from there (Thanks to Andrew Dupont for the technique)
-			// IE 8 doesn't work on object elements
-			if ( nodeType === 1 && context.nodeName.toLowerCase() !== "object" ) {
-				groups = tokenize( selector );
-
-				if ( (old = context.getAttribute("id")) ) {
-					nid = old.replace( rescape, "\\$&" );
-				} else {
-					context.setAttribute( "id", nid );
-				}
-				nid = "[id='" + nid + "'] ";
-
-				i = groups.length;
-				while ( i-- ) {
-					groups[i] = nid + toSelector( groups[i] );
-				}
-				newContext = rsibling.test( selector ) && testContext( context.parentNode ) || context;
-				newSelector = groups.join(",");
-			}
-
-			if ( newSelector ) {
-				try {
-					push.apply( results,
-						newContext.querySelectorAll( newSelector )
-					);
-					return results;
-				} catch(qsaError) {
-				} finally {
-					if ( !old ) {
-						context.removeAttribute("id");
-					}
-				}
-			}
-		}
-	}
-
-	// All others
-	return select( selector.replace( rtrim, "$1" ), context, results, seed );
-}
-
-/**
- * Create key-value caches of limited size
- * @returns {Function(string, Object)} Returns the Object data after storing it on itself with
- *	property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
- *	deleting the oldest entry
- */
-function createCache() {
-	var keys = [];
-
-	function cache( key, value ) {
-		// Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
-		if ( keys.push( key + " " ) > Expr.cacheLength ) {
-			// Only keep the most recent entries
-			delete cache[ keys.shift() ];
-		}
-		return (cache[ key + " " ] = value);
-	}
-	return cache;
-}
-
-/**
- * Mark a function for special use by Sizzle
- * @param {Function} fn The function to mark
- */
-function markFunction( fn ) {
-	fn[ expando ] = true;
-	return fn;
-}
-
-/**
- * Support testing using an element
- * @param {Function} fn Passed the created div and expects a boolean result
- */
-function assert( fn ) {
-	var div = document.createElement("div");
-
-	try {
-		return !!fn( div );
-	} catch (e) {
-		return false;
-	} finally {
-		// Remove from its parent by default
-		if ( div.parentNode ) {
-			div.parentNode.removeChild( div );
-		}
-		// release memory in IE
-		div = null;
-	}
-}
-
-/**
- * Adds the same handler for all of the specified attrs
- * @param {String} attrs Pipe-separated list of attributes
- * @param {Function} handler The method that will be applied
- */
-function addHandle( attrs, handler ) {
-	var arr = attrs.split("|"),
-		i = attrs.length;
-
-	while ( i-- ) {
-		Expr.attrHandle[ arr[i] ] = handler;
-	}
-}
-
-/**
- * Checks document order of two siblings
- * @param {Element} a
- * @param {Element} b
- * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
- */
-function siblingCheck( a, b ) {
-	var cur = b && a,
-		diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
-			( ~b.sourceIndex || MAX_NEGATIVE ) -
-			( ~a.sourceIndex || MAX_NEGATIVE );
-
-	// Use IE sourceIndex if available on both nodes
-	if ( diff ) {
-		return diff;
-	}
-
-	// Check if b follows a
-	if ( cur ) {
-		while ( (cur = cur.nextSibling) ) {
-			if ( cur === b ) {
-				return -1;
-			}
-		}
-	}
-
-	return a ? 1 : -1;
-}
-
-/**
- * Returns a function to use in pseudos for input types
- * @param {String} type
- */
-function createInputPseudo( type ) {
-	return function( elem ) {
-		var name = elem.nodeName.toLowerCase();
-		return name === "input" && elem.type === type;
-	};
-}
-
-/**
- * Returns a function to use in pseudos for buttons
- * @param {String} type
- */
-function createButtonPseudo( type ) {
-	return function( elem ) {
-		var name = elem.nodeName.toLowerCase();
-		return (name === "input" || name === "button") && elem.type === type;
-	};
-}
-
-/**
- * Returns a function to use in pseudos for positionals
- * @param {Function} fn
- */
-function createPositionalPseudo( fn ) {
-	return markFunction(function( argument ) {
-		argument = +argument;
-		return markFunction(function( seed, matches ) {
-			var j,
-				matchIndexes = fn( [], seed.length, argument ),
-				i = matchIndexes.length;
-
-			// Match elements found at the specified indexes
-			while ( i-- ) {
-				if ( seed[ (j = matchIndexes[i]) ] ) {
-					seed[j] = !(matches[j] = seed[j]);
-				}
-			}
-		});
-	});
-}
-
-/**
- * Checks a node for validity as a Sizzle context
- * @param {Element|Object=} context
- * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
- */
-function testContext( context ) {
-	return context && typeof context.getElementsByTagName !== "undefined" && context;
-}
-
-// Expose support vars for convenience
-support = Sizzle.support = {};
-
-/**
- * Detects XML nodes
- * @param {Element|Object} elem An element or a document
- * @returns {Boolean} True iff elem is a non-HTML XML node
- */
-isXML = Sizzle.isXML = function( elem ) {
-	// documentElement is verified for cases where it doesn't yet exist
-	// (such as loading iframes in IE - #4833)
-	var documentElement = elem && (elem.ownerDocument || elem).documentElement;
-	return documentElement ? documentElement.nodeName !== "HTML" : false;
-};
-
-/**
- * Sets document-related variables once based on the current document
- * @param {Element|Object} [doc] An element or document object to use to set the document
- * @returns {Object} Returns the current document
- */
-setDocument = Sizzle.setDocument = function( node ) {
-	var hasCompare, parent,
-		doc = node ? node.ownerDocument || node : preferredDoc;
-
-	// If no document and documentElement is available, return
-	if ( doc === document || doc.nodeType !== 9 || !doc.documentElement ) {
-		return document;
-	}
-
-	// Set our document
-	document = doc;
-	docElem = doc.documentElement;
-	parent = doc.defaultView;
-
-	// Support: IE>8
-	// If iframe document is assigned to "document" variable and if iframe has been reloaded,
-	// IE will throw "permission denied" error when accessing "document" variable, see jQuery #13936
-	// IE6-8 do not support the defaultView property so parent will be undefined
-	if ( parent && parent !== parent.top ) {
-		// IE11 does not have attachEvent, so all must suffer
-		if ( parent.addEventListener ) {
-			parent.addEventListener( "unload", unloadHandler, false );
-		} else if ( parent.attachEvent ) {
-			parent.attachEvent( "onunload", unloadHandler );
-		}
-	}
-
-	/* Support tests
-	---------------------------------------------------------------------- */
-	documentIsHTML = !isXML( doc );
-
-	/* Attributes
-	---------------------------------------------------------------------- */
-
-	// Support: IE<8
-	// Verify that getAttribute really returns attributes and not properties
-	// (excepting IE8 booleans)
-	support.attributes = assert(function( div ) {
-		div.className = "i";
-		return !div.getAttribute("className");
-	});
-
-	/* getElement(s)By*
-	---------------------------------------------------------------------- */
-
-	// Check if getElementsByTagName("*") returns only elements
-	support.getElementsByTagName = assert(function( div ) {
-		div.appendChild( doc.createComment("") );
-		return !div.getElementsByTagName("*").length;
-	});
-
-	// Support: IE<9
-	support.getElementsByClassName = rnative.test( doc.getElementsByClassName );
-
-	// Support: IE<10
-	// Check if getElementById returns elements by name
-	// The broken getElementById methods don't pick up programatically-set names,
-	// so use a roundabout getElementsByName test
-	support.getById = assert(function( div ) {
-		docElem.appendChild( div ).id = expando;
-		return !doc.getElementsByName || !doc.getElementsByName( expando ).length;
-	});
-
-	// ID find and filter
-	if ( support.getById ) {
-		Expr.find["ID"] = function( id, context ) {
-			if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
-				var m = context.getElementById( id );
-				// Check parentNode to catch when Blackberry 4.6 returns
-				// nodes that are no longer in the document #6963
-				return m && m.parentNode ? [ m ] : [];
-			}
-		};
-		Expr.filter["ID"] = function( id ) {
-			var attrId = id.replace( runescape, funescape );
-			return function( elem ) {
-				return elem.getAttribute("id") === attrId;
-			};
-		};
-	} else {
-		// Support: IE6/7
-		// getElementById is not reliable as a find shortcut
-		delete Expr.find["ID"];
-
-		Expr.filter["ID"] =  function( id ) {
-			var attrId = id.replace( runescape, funescape );
-			return function( elem ) {
-				var node = typeof elem.getAttributeNode !== "undefined" && elem.getAttributeNode("id");
-				return node && node.value === attrId;
-			};
-		};
-	}
-
-	// Tag
-	Expr.find["TAG"] = support.getElementsByTagName ?
-		function( tag, context ) {
-			if ( typeof context.getElementsByTagName !== "undefined" ) {
-				return context.getElementsByTagName( tag );
-
-			// DocumentFragment nodes don't have gEBTN
-			} else if ( support.qsa ) {
-				return context.querySelectorAll( tag );
-			}
-		} :
-
-		function( tag, context ) {
-			var elem,
-				tmp = [],
-				i = 0,
-				// By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
-				results = context.getElementsByTagName( tag );
-
-			// Filter out possible comments
-			if ( tag === "*" ) {
-				while ( (elem = results[i++]) ) {
-					if ( elem.nodeType === 1 ) {
-						tmp.push( elem );
-					}
-				}
-
-				return tmp;
-			}
-			return results;
-		};
-
-	// Class
-	Expr.find["CLASS"] = support.getElementsByClassName && function( className, context ) {
-		if ( documentIsHTML ) {
-			return context.getElementsByClassName( className );
-		}
-	};
-
-	/* QSA/matchesSelector
-	---------------------------------------------------------------------- */
-
-	// QSA and matchesSelector support
-
-	// matchesSelector(:active) reports false when true (IE9/Opera 11.5)
-	rbuggyMatches = [];
-
-	// qSa(:focus) reports false when true (Chrome 21)
-	// We allow this because of a bug in IE8/9 that throws an error
-	// whenever `document.activeElement` is accessed on an iframe
-	// So, we allow :focus to pass through QSA all the time to avoid the IE error
-	// See http://bugs.jquery.com/ticket/13378
-	rbuggyQSA = [];
-
-	if ( (support.qsa = rnative.test( doc.querySelectorAll )) ) {
-		// Build QSA regex
-		// Regex strategy adopted from Diego Perini
-		assert(function( div ) {
-			// Select is set to empty string on purpose
-			// This is to test IE's treatment of not explicitly
-			// setting a boolean content attribute,
-			// since its presence should be enough
-			// http://bugs.jquery.com/ticket/12359
-			docElem.appendChild( div ).innerHTML = "<a id='" + expando + "'></a>" +
-				"<select id='" + expando + "-\f]' msallowcapture=''>" +
-				"<option selected=''></option></select>";
-
-			// Support: IE8, Opera 11-12.16
-			// Nothing should be selected when empty strings follow ^= or $= or *=
-			// The test attribute must be unknown in Opera but "safe" for WinRT
-			// http://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
-			if ( div.querySelectorAll("[msallowcapture^='']").length ) {
-				rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
-			}
-
-			// Support: IE8
-			// Boolean attributes and "value" are not treated correctly
-			if ( !div.querySelectorAll("[selected]").length ) {
-				rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
-			}
-
-			// Support: Chrome<29, Android<4.2+, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.7+
-			if ( !div.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
-				rbuggyQSA.push("~=");
-			}
-
-			// Webkit/Opera - :checked should return selected option elements
-			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
-			// IE8 throws error here and will not see later tests
-			if ( !div.querySelectorAll(":checked").length ) {
-				rbuggyQSA.push(":checked");
-			}
-
-			// Support: Safari 8+, iOS 8+
-			// https://bugs.webkit.org/show_bug.cgi?id=136851
-			// In-page `selector#id sibing-combinator selector` fails
-			if ( !div.querySelectorAll( "a#" + expando + "+*" ).length ) {
-				rbuggyQSA.push(".#.+[+~]");
-			}
-		});
-
-		assert(function( div ) {
-			// Support: Windows 8 Native Apps
-			// The type and name attributes are restricted during .innerHTML assignment
-			var input = doc.createElement("input");
-			input.setAttribute( "type", "hidden" );
-			div.appendChild( input ).setAttribute( "name", "D" );
-
-			// Support: IE8
-			// Enforce case-sensitivity of name attribute
-			if ( div.querySelectorAll("[name=d]").length ) {
-				rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
-			}
-
-			// FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
-			// IE8 throws error here and will not see later tests
-			if ( !div.querySelectorAll(":enabled").length ) {
-				rbuggyQSA.push( ":enabled", ":disabled" );
-			}
-
-			// Opera 10-11 does not throw on post-comma invalid pseudos
-			div.querySelectorAll("*,:x");
-			rbuggyQSA.push(",.*:");
-		});
-	}
-
-	if ( (support.matchesSelector = rnative.test( (matches = docElem.matches ||
-		docElem.webkitMatchesSelector ||
-		docElem.mozMatchesSelector ||
-		docElem.oMatchesSelector ||
-		docElem.msMatchesSelector) )) ) {
-
-		assert(function( div ) {
-			// Check to see if it's possible to do matchesSelector
-			// on a disconnected node (IE 9)
-			support.disconnectedMatch = matches.call( div, "div" );
-
-			// This should fail with an exception
-			// Gecko does not error, returns false instead
-			matches.call( div, "[s!='']:x" );
-			rbuggyMatches.push( "!=", pseudos );
-		});
-	}
-
-	rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join("|") );
-	rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join("|") );
-
-	/* Contains
-	---------------------------------------------------------------------- */
-	hasCompare = rnative.test( docElem.compareDocumentPosition );
-
-	// Element contains another
-	// Purposefully does not implement inclusive descendent
-	// As in, an element does not contain itself
-	contains = hasCompare || rnative.test( docElem.contains ) ?
-		function( a, b ) {
-			var adown = a.nodeType === 9 ? a.documentElement : a,
-				bup = b && b.parentNode;
-			return a === bup || !!( bup && bup.nodeType === 1 && (
-				adown.contains ?
-					adown.contains( bup ) :
-					a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
-			));
-		} :
-		function( a, b ) {
-			if ( b ) {
-				while ( (b = b.parentNode) ) {
-					if ( b === a ) {
-						return true;
-					}
-				}
-			}
-			return false;
-		};
-
-	/* Sorting
-	---------------------------------------------------------------------- */
-
-	// Document order sorting
-	sortOrder = hasCompare ?
-	function( a, b ) {
-
-		// Flag for duplicate removal
-		if ( a === b ) {
-			hasDuplicate = true;
-			return 0;
-		}
-
-		// Sort on method existence if only one input has compareDocumentPosition
-		var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
-		if ( compare ) {
-			return compare;
-		}
-
-		// Calculate position if both inputs belong to the same document
-		compare = ( a.ownerDocument || a ) === ( b.ownerDocument || b ) ?
-			a.compareDocumentPosition( b ) :
-
-			// Otherwise we know they are disconnected
-			1;
-
-		// Disconnected nodes
-		if ( compare & 1 ||
-			(!support.sortDetached && b.compareDocumentPosition( a ) === compare) ) {
-
-			// Choose the first element that is related to our preferred document
-			if ( a === doc || a.ownerDocument === preferredDoc && contains(preferredDoc, a) ) {
-				return -1;
-			}
-			if ( b === doc || b.ownerDocument === preferredDoc && contains(preferredDoc, b) ) {
-				return 1;
-			}
-
-			// Maintain original order
-			return sortInput ?
-				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
-				0;
-		}
-
-		return compare & 4 ? -1 : 1;
-	} :
-	function( a, b ) {
-		// Exit early if the nodes are identical
-		if ( a === b ) {
-			hasDuplicate = true;
-			return 0;
-		}
-
-		var cur,
-			i = 0,
-			aup = a.parentNode,
-			bup = b.parentNode,
-			ap = [ a ],
-			bp = [ b ];
-
-		// Parentless nodes are either documents or disconnected
-		if ( !aup || !bup ) {
-			return a === doc ? -1 :
-				b === doc ? 1 :
-				aup ? -1 :
-				bup ? 1 :
-				sortInput ?
-				( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
-				0;
-
-		// If the nodes are siblings, we can do a quick check
-		} else if ( aup === bup ) {
-			return siblingCheck( a, b );
-		}
-
-		// Otherwise we need full lists of their ancestors for comparison
-		cur = a;
-		while ( (cur = cur.parentNode) ) {
-			ap.unshift( cur );
-		}
-		cur = b;
-		while ( (cur = cur.parentNode) ) {
-			bp.unshift( cur );
-		}
-
-		// Walk down the tree looking for a discrepancy
-		while ( ap[i] === bp[i] ) {
-			i++;
-		}
-
-		return i ?
-			// Do a sibling check if the nodes have a common ancestor
-			siblingCheck( ap[i], bp[i] ) :
-
-			// Otherwise nodes in our document sort first
-			ap[i] === preferredDoc ? -1 :
-			bp[i] === preferredDoc ? 1 :
-			0;
-	};
-
-	return doc;
-};
-
-Sizzle.matches = function( expr, elements ) {
-	return Sizzle( expr, null, null, elements );
-};
-
-Sizzle.matchesSelector = function( elem, expr ) {
-	// Set document vars if needed
-	if ( ( elem.ownerDocument || elem ) !== document ) {
-		setDocument( elem );
-	}
-
-	// Make sure that attribute selectors are quoted
-	expr = expr.replace( rattributeQuotes, "='$1']" );
-
-	if ( support.matchesSelector && documentIsHTML &&
-		( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
-		( !rbuggyQSA     || !rbuggyQSA.test( expr ) ) ) {
-
-		try {
-			var ret = matches.call( elem, expr );
-
-			// IE 9's matchesSelector returns false on disconnected nodes
-			if ( ret || support.disconnectedMatch ||
-					// As well, disconnected nodes are said to be in a document
-					// fragment in IE 9
-					elem.document && elem.document.nodeType !== 11 ) {
-				return ret;
-			}
-		} catch (e) {}
-	}
-
-	return Sizzle( expr, document, null, [ elem ] ).length > 0;
-};
-
-Sizzle.contains = function( context, elem ) {
-	// Set document vars if needed
-	if ( ( context.ownerDocument || context ) !== document ) {
-		setDocument( context );
-	}
-	return contains( context, elem );
-};
-
-Sizzle.attr = function( elem, name ) {
-	// Set document vars if needed
-	if ( ( elem.ownerDocument || elem ) !== document ) {
-		setDocument( elem );
-	}
-
-	var fn = Expr.attrHandle[ name.toLowerCase() ],
-		// Don't get fooled by Object.prototype properties (jQuery #13807)
-		val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
-			fn( elem, name, !documentIsHTML ) :
-			undefined;
-
-	return val !== undefined ?
-		val :
-		support.attributes || !documentIsHTML ?
-			elem.getAttribute( name ) :
-			(val = elem.getAttributeNode(name)) && val.specified ?
-				val.value :
-				null;
-};
-
-Sizzle.error = function( msg ) {
-	throw new Error( "Syntax error, unrecognized expression: " + msg );
-};
-
-/**
- * Document sorting and removing duplicates
- * @param {ArrayLike} results
- */
-Sizzle.uniqueSort = function( results ) {
-	var elem,
-		duplicates = [],
-		j = 0,
-		i = 0;
-
-	// Unless we *know* we can detect duplicates, assume their presence
-	hasDuplicate = !support.detectDuplicates;
-	sortInput = !support.sortStable && results.slice( 0 );
-	results.sort( sortOrder );
-
-	if ( hasDuplicate ) {
-		while ( (elem = results[i++]) ) {
-			if ( elem === results[ i ] ) {
-				j = duplicates.push( i );
-			}
-		}
-		while ( j-- ) {
-			results.splice( duplicates[ j ], 1 );
-		}
-	}
-
-	// Clear input after sorting to release objects
-	// See https://github.com/jquery/sizzle/pull/225
-	sortInput = null;
-
-	return results;
-};
-
-/**
- * Utility function for retrieving the text value of an array of DOM nodes
- * @param {Array|Element} elem
- */
-getText = Sizzle.getText = function( elem ) {
-	var node,
-		ret = "",
-		i = 0,
-		nodeType = elem.nodeType;
-
-	if ( !nodeType ) {
-		// If no nodeType, this is expected to be an array
-		while ( (node = elem[i++]) ) {
-			// Do not traverse comment nodes
-			ret += getText( node );
-		}
-	} else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
-		// Use textContent for elements
-		// innerText usage removed for consistency of new lines (jQuery #11153)
-		if ( typeof elem.textContent === "string" ) {
-			return elem.textContent;
-		} else {
-			// Traverse its children
-			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
-				ret += getText( elem );
-			}
-		}
-	} else if ( nodeType === 3 || nodeType === 4 ) {
-		return elem.nodeValue;
-	}
-	// Do not include comment or processing instruction nodes
-
-	return ret;
-};
-
-Expr = Sizzle.selectors = {
-
-	// Can be adjusted by the user
-	cacheLength: 50,
-
-	createPseudo: markFunction,
-
-	match: matchExpr,
-
-	attrHandle: {},
-
-	find: {},
-
-	relative: {
-		">": { dir: "parentNode", first: true },
-		" ": { dir: "parentNode" },
-		"+": { dir: "previousSibling", first: true },
-		"~": { dir: "previousSibling" }
-	},
-
-	preFilter: {
-		"ATTR": function( match ) {
-			match[1] = match[1].replace( runescape, funescape );
-
-			// Move the given value to match[3] whether quoted or unquoted
-			match[3] = ( match[3] || match[4] || match[5] || "" ).replace( runescape, funescape );
-
-			if ( match[2] === "~=" ) {
-				match[3] = " " + match[3] + " ";
-			}
-
-			return match.slice( 0, 4 );
-		},
-
-		"CHILD": function( match ) {
-			/* matches from matchExpr["CHILD"]
-				1 type (only|nth|...)
-				2 what (child|of-type)
-				3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
-				4 xn-component of xn+y argument ([+-]?\d*n|)
-				5 sign of xn-component
-				6 x of xn-component
-				7 sign of y-component
-				8 y of y-component
-			*/
-			match[1] = match[1].toLowerCase();
-
-			if ( match[1].slice( 0, 3 ) === "nth" ) {
-				// nth-* requires argument
-				if ( !match[3] ) {
-					Sizzle.error( match[0] );
-				}
-
-				// numeric x and y parameters for Expr.filter.CHILD
-				// remember that false/true cast respectively to 0/1
-				match[4] = +( match[4] ? match[5] + (match[6] || 1) : 2 * ( match[3] === "even" || match[3] === "odd" ) );
-				match[5] = +( ( match[7] + match[8] ) || match[3] === "odd" );
-
-			// other types prohibit arguments
-			} else if ( match[3] ) {
-				Sizzle.error( match[0] );
-			}
-
-			return match;
-		},
-
-		"PSEUDO": function( match ) {
-			var excess,
-				unquoted = !match[6] && match[2];
-
-			if ( matchExpr["CHILD"].test( match[0] ) ) {
-				return null;
-			}
-
-			// Accept quoted arguments as-is
-			if ( match[3] ) {
-				match[2] = match[4] || match[5] || "";
-
-			// Strip excess characters from unquoted arguments
-			} else if ( unquoted && rpseudo.test( unquoted ) &&
-				// Get excess from tokenize (recursively)
-				(excess = tokenize( unquoted, true )) &&
-				// advance to the next closing parenthesis
-				(excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length) ) {
-
-				// excess is a negative index
-				match[0] = match[0].slice( 0, excess );
-				match[2] = unquoted.slice( 0, excess );
-			}
-
-			// Return only captures needed by the pseudo filter method (type and argument)
-			return match.slice( 0, 3 );
-		}
-	},
-
-	filter: {
-
-		"TAG": function( nodeNameSelector ) {
-			var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
-			return nodeNameSelector === "*" ?
-				function() { return true; } :
-				function( elem ) {
-					return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
-				};
-		},
-
-		"CLASS": function( className ) {
-			var pattern = classCache[ className + " " ];
-
-			return pattern ||
-				(pattern = new RegExp( "(^|" + whitespace + ")" + className + "(" + whitespace + "|$)" )) &&
-				classCache( className, function( elem ) {
-					return pattern.test( typeof elem.className === "string" && elem.className || typeof elem.getAttribute !== "undefined" && elem.getAttribute("class") || "" );
-				});
-		},
-
-		"ATTR": function( name, operator, check ) {
-			return function( elem ) {
-				var result = Sizzle.attr( elem, name );
-
-				if ( result == null ) {
-					return operator === "!=";
-				}
-				if ( !operator ) {
-					return true;
-				}
-
-				result += "";
-
-				return operator === "=" ? result === check :
-					operator === "!=" ? result !== check :
-					operator === "^=" ? check && result.indexOf( check ) === 0 :
-					operator === "*=" ? check && result.indexOf( check ) > -1 :
-					operator === "$=" ? check && result.slice( -check.length ) === check :
-					operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
-					operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
-					false;
-			};
-		},
-
-		"CHILD": function( type, what, argument, first, last ) {
-			var simple = type.slice( 0, 3 ) !== "nth",
-				forward = type.slice( -4 ) !== "last",
-				ofType = what === "of-type";
-
-			return first === 1 && last === 0 ?
-
-				// Shortcut for :nth-*(n)
-				function( elem ) {
-					return !!elem.parentNode;
-				} :
-
-				function( elem, context, xml ) {
-					var cache, outerCache, node, diff, nodeIndex, start,
-						dir = simple !== forward ? "nextSibling" : "previousSibling",
-						parent = elem.parentNode,
-						name = ofType && elem.nodeName.toLowerCase(),
-						useCache = !xml && !ofType;
-
-					if ( parent ) {
-
-						// :(first|last|only)-(child|of-type)
-						if ( simple ) {
-							while ( dir ) {
-								node = elem;
-								while ( (node = node[ dir ]) ) {
-									if ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) {
-										return false;
-									}
-								}
-								// Reverse direction for :only-* (if we haven't yet done so)
-								start = dir = type === "only" && !start && "nextSibling";
-							}
-							return true;
-						}
-
-						start = [ forward ? parent.firstChild : parent.lastChild ];
-
-						// non-xml :nth-child(...) stores cache data on `parent`
-						if ( forward && useCache ) {
-							// Seek `elem` from a previously-cached index
-							outerCache = parent[ expando ] || (parent[ expando ] = {});
-							cache = outerCache[ type ] || [];
-							nodeIndex = cache[0] === dirruns && cache[1];
-							diff = cache[0] === dirruns && cache[2];
-							node = nodeIndex && parent.childNodes[ nodeIndex ];
-
-							while ( (node = ++nodeIndex && node && node[ dir ] ||
-
-								// Fallback to seeking `elem` from the start
-								(diff = nodeIndex = 0) || start.pop()) ) {
-
-								// When found, cache indexes on `parent` and break
-								if ( node.nodeType === 1 && ++diff && node === elem ) {
-									outerCache[ type ] = [ dirruns, nodeIndex, diff ];
-									break;
-								}
-							}
-
-						// Use previously-cached element index if available
-						} else if ( useCache && (cache = (elem[ expando ] || (elem[ expando ] = {}))[ type ]) && cache[0] === dirruns ) {
-							diff = cache[1];
-
-						// xml :nth-child(...) or :nth-last-child(...) or :nth(-last)?-of-type(...)
-						} else {
-							// Use the same loop as above to seek `elem` from the start
-							while ( (node = ++nodeIndex && node && node[ dir ] ||
-								(diff = nodeIndex = 0) || start.pop()) ) {
-
-								if ( ( ofType ? node.nodeName.toLowerCase() === name : node.nodeType === 1 ) && ++diff ) {
-									// Cache the index of each encountered element
-									if ( useCache ) {
-										(node[ expando ] || (node[ expando ] = {}))[ type ] = [ dirruns, diff ];
-									}
-
-									if ( node === elem ) {
-										break;
-									}
-								}
-							}
-						}
-
-						// Incorporate the offset, then check against cycle size
-						diff -= last;
-						return diff === first || ( diff % first === 0 && diff / first >= 0 );
-					}
-				};
-		},
-
-		"PSEUDO": function( pseudo, argument ) {
-			// pseudo-class names are case-insensitive
-			// http://www.w3.org/TR/selectors/#pseudo-classes
-			// Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
-			// Remember that setFilters inherits from pseudos
-			var args,
-				fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
-					Sizzle.error( "unsupported pseudo: " + pseudo );
-
-			// The user may use createPseudo to indicate that
-			// arguments are needed to create the filter function
-			// just as Sizzle does
-			if ( fn[ expando ] ) {
-				return fn( argument );
-			}
-
-			// But maintain support for old signatures
-			if ( fn.length > 1 ) {
-				args = [ pseudo, pseudo, "", argument ];
-				return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
-					markFunction(function( seed, matches ) {
-						var idx,
-							matched = fn( seed, argument ),
-							i = matched.length;
-						while ( i-- ) {
-							idx = indexOf( seed, matched[i] );
-							seed[ idx ] = !( matches[ idx ] = matched[i] );
-						}
-					}) :
-					function( elem ) {
-						return fn( elem, 0, args );
-					};
-			}
-
-			return fn;
-		}
-	},
-
-	pseudos: {
-		// Potentially complex pseudos
-		"not": markFunction(function( selector ) {
-			// Trim the selector passed to compile
-			// to avoid treating leading and trailing
-			// spaces as combinators
-			var input = [],
-				results = [],
-				matcher = compile( selector.replace( rtrim, "$1" ) );
-
-			return matcher[ expando ] ?
-				markFunction(function( seed, matches, context, xml ) {
-					var elem,
-						unmatched = matcher( seed, null, xml, [] ),
-						i = seed.length;
-
-					// Match elements unmatched by `matcher`
-					while ( i-- ) {
-						if ( (elem = unmatched[i]) ) {
-							seed[i] = !(matches[i] = elem);
-						}
-					}
-				}) :
-				function( elem, context, xml ) {
-					input[0] = elem;
-					matcher( input, null, xml, results );
-					// Don't keep the element (issue #299)
-					input[0] = null;
-					return !results.pop();
-				};
-		}),
-
-		"has": markFunction(function( selector ) {
-			return function( elem ) {
-				return Sizzle( selector, elem ).length > 0;
-			};
-		}),
-
-		"contains": markFunction(function( text ) {
-			text = text.replace( runescape, funescape );
-			return function( elem ) {
-				return ( elem.textContent || elem.innerText || getText( elem ) ).indexOf( text ) > -1;
-			};
-		}),
-
-		// "Whether an element is represented by a :lang() selector
-		// is based solely on the element's language value
-		// being equal to the identifier C,
-		// or beginning with the identifier C immediately followed by "-".
-		// The matching of C against the element's language value is performed case-insensitively.
-		// The identifier C does not have to be a valid language name."
-		// http://www.w3.org/TR/selectors/#lang-pseudo
-		"lang": markFunction( function( lang ) {
-			// lang value must be a valid identifier
-			if ( !ridentifier.test(lang || "") ) {
-				Sizzle.error( "unsupported lang: " + lang );
-			}
-			lang = lang.replace( runescape, funescape ).toLowerCase();
-			return function( elem ) {
-				var elemLang;
-				do {
-					if ( (elemLang = documentIsHTML ?
-						elem.lang :
-						elem.getAttribute("xml:lang") || elem.getAttribute("lang")) ) {
-
-						elemLang = elemLang.toLowerCase();
-						return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
-					}
-				} while ( (elem = elem.parentNode) && elem.nodeType === 1 );
-				return false;
-			};
-		}),
-
-		// Miscellaneous
-		"target": function( elem ) {
-			var hash = window.location && window.location.hash;
-			return hash && hash.slice( 1 ) === elem.id;
-		},
-
-		"root": function( elem ) {
-			return elem === docElem;
-		},
-
-		"focus": function( elem ) {
-			return elem === document.activeElement && (!document.hasFocus || document.hasFocus()) && !!(elem.type || elem.href || ~elem.tabIndex);
-		},
-
-		// Boolean properties
-		"enabled": function( elem ) {
-			return elem.disabled === false;
-		},
-
-		"disabled": function( elem ) {
-			return elem.disabled === true;
-		},
-
-		"checked": function( elem ) {
-			// In CSS3, :checked should return both checked and selected elements
-			// http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
-			var nodeName = elem.nodeName.toLowerCase();
-			return (nodeName === "input" && !!elem.checked) || (nodeName === "option" && !!elem.selected);
-		},
-
-		"selected": function( elem ) {
-			// Accessing this property makes selected-by-default
-			// options in Safari work properly
-			if ( elem.parentNode ) {
-				elem.parentNode.selectedIndex;
-			}
-
-			return elem.selected === true;
-		},
-
-		// Contents
-		"empty": function( elem ) {
-			// http://www.w3.org/TR/selectors/#empty-pseudo
-			// :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
-			//   but not by others (comment: 8; processing instruction: 7; etc.)
-			// nodeType < 6 works because attributes (2) do not appear as children
-			for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
-				if ( elem.nodeType < 6 ) {
-					return false;
-				}
-			}
-			return true;
-		},
-
-		"parent": function( elem ) {
-			return !Expr.pseudos["empty"]( elem );
-		},
-
-		// Element/input types
-		"header": function( elem ) {
-			return rheader.test( elem.nodeName );
-		},
-
-		"input": function( elem ) {
-			return rinputs.test( elem.nodeName );
-		},
-
-		"button": function( elem ) {
-			var name = elem.nodeName.toLowerCase();
-			return name === "input" && elem.type === "button" || name === "button";
-		},
-
-		"text": function( elem ) {
-			var attr;
-			return elem.nodeName.toLowerCase() === "input" &&
-				elem.type === "text" &&
-
-				// Support: IE<8
-				// New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
-				( (attr = elem.getAttribute("type")) == null || attr.toLowerCase() === "text" );
-		},
-
-		// Position-in-collection
-		"first": createPositionalPseudo(function() {
-			return [ 0 ];
-		}),
-
-		"last": createPositionalPseudo(function( matchIndexes, length ) {
-			return [ length - 1 ];
-		}),
-
-		"eq": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			return [ argument < 0 ? argument + length : argument ];
-		}),
-
-		"even": createPositionalPseudo(function( matchIndexes, length ) {
-			var i = 0;
-			for ( ; i < length; i += 2 ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"odd": createPositionalPseudo(function( matchIndexes, length ) {
-			var i = 1;
-			for ( ; i < length; i += 2 ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"lt": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			var i = argument < 0 ? argument + length : argument;
-			for ( ; --i >= 0; ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		}),
-
-		"gt": createPositionalPseudo(function( matchIndexes, length, argument ) {
-			var i = argument < 0 ? argument + length : argument;
-			for ( ; ++i < length; ) {
-				matchIndexes.push( i );
-			}
-			return matchIndexes;
-		})
-	}
-};
-
-Expr.pseudos["nth"] = Expr.pseudos["eq"];
-
-// Add button/input type pseudos
-for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
-	Expr.pseudos[ i ] = createInputPseudo( i );
-}
-for ( i in { submit: true, reset: true } ) {
-	Expr.pseudos[ i ] = createButtonPseudo( i );
-}
-
-// Easy API for creating new setFilters
-function setFilters() {}
-setFilters.prototype = Expr.filters = Expr.pseudos;
-Expr.setFilters = new setFilters();
-
-tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
-	var matched, match, tokens, type,
-		soFar, groups, preFilters,
-		cached = tokenCache[ selector + " " ];
-
-	if ( cached ) {
-		return parseOnly ? 0 : cached.slice( 0 );
-	}
-
-	soFar = selector;
-	groups = [];
-	preFilters = Expr.preFilter;
-
-	while ( soFar ) {
-
-		// Comma and first run
-		if ( !matched || (match = rcomma.exec( soFar )) ) {
-			if ( match ) {
-				// Don't consume trailing commas as valid
-				soFar = soFar.slice( match[0].length ) || soFar;
-			}
-			groups.push( (tokens = []) );
-		}
-
-		matched = false;
-
-		// Combinators
-		if ( (match = rcombinators.exec( soFar )) ) {
-			matched = match.shift();
-			tokens.push({
-				value: matched,
-				// Cast descendant combinators to space
-				type: match[0].replace( rtrim, " " )
-			});
-			soFar = soFar.slice( matched.length );
-		}
-
-		// Filters
-		for ( type in Expr.filter ) {
-			if ( (match = matchExpr[ type ].exec( soFar )) && (!preFilters[ type ] ||
-				(match = preFilters[ type ]( match ))) ) {
-				matched = match.shift();
-				tokens.push({
-					value: matched,
-					type: type,
-					matches: match
-				});
-				soFar = soFar.slice( matched.length );
-			}
-		}
-
-		if ( !matched ) {
-			break;
-		}
-	}
-
-	// Return the length of the invalid excess
-	// if we're just parsing
-	// Otherwise, throw an error or return tokens
-	return parseOnly ?
-		soFar.length :
-		soFar ?
-			Sizzle.error( selector ) :
-			// Cache the tokens
-			tokenCache( selector, groups ).slice( 0 );
-};
-
-function toSelector( tokens ) {
-	var i = 0,
-		len = tokens.length,
-		selector = "";
-	for ( ; i < len; i++ ) {
-		selector += tokens[i].value;
-	}
-	return selector;
-}
-
-function addCombinator( matcher, combinator, base ) {
-	var dir = combinator.dir,
-		checkNonElements = base && dir === "parentNode",
-		doneName = done++;
-
-	return combinator.first ?
-		// Check against closest ancestor/preceding element
-		function( elem, context, xml ) {
-			while ( (elem = elem[ dir ]) ) {
-				if ( elem.nodeType === 1 || checkNonElements ) {
-					return matcher( elem, context, xml );
-				}
-			}
-		} :
-
-		// Check against all ancestor/preceding elements
-		function( elem, context, xml ) {
-			var oldCache, outerCache,
-				newCache = [ dirruns, doneName ];
-
-			// We can't set arbitrary data on XML nodes, so they don't benefit from dir caching
-			if ( xml ) {
-				while ( (elem = elem[ dir ]) ) {
-					if ( elem.nodeType === 1 || checkNonElements ) {
-						if ( matcher( elem, context, xml ) ) {
-							return true;
-						}
-					}
-				}
-			} else {
-				while ( (elem = elem[ dir ]) ) {
-					if ( elem.nodeType === 1 || checkNonElements ) {
-						outerCache = elem[ expando ] || (elem[ expando ] = {});
-						if ( (oldCache = outerCache[ dir ]) &&
-							oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
-
-							// Assign to newCache so results back-propagate to previous elements
-							return (newCache[ 2 ] = oldCache[ 2 ]);
-						} else {
-							// Reuse newcache so results back-propagate to previous elements
-							outerCache[ dir ] = newCache;
-
-							// A match means we're done; a fail means we have to keep checking
-							if ( (newCache[ 2 ] = matcher( elem, context, xml )) ) {
-								return true;
-							}
-						}
-					}
-				}
-			}
-		};
-}
-
-function elementMatcher( matchers ) {
-	return matchers.length > 1 ?
-		function( elem, context, xml ) {
-			var i = matchers.length;
-			while ( i-- ) {
-				if ( !matchers[i]( elem, context, xml ) ) {
-					return false;
-				}
-			}
-			return true;
-		} :
-		matchers[0];
-}
-
-function multipleContexts( selector, contexts, results ) {
-	var i = 0,
-		len = contexts.length;
-	for ( ; i < len; i++ ) {
-		Sizzle( selector, contexts[i], results );
-	}
-	return results;
-}
-
-function condense( unmatched, map, filter, context, xml ) {
-	var elem,
-		newUnmatched = [],
-		i = 0,
-		len = unmatched.length,
-		mapped = map != null;
-
-	for ( ; i < len; i++ ) {
-		if ( (elem = unmatched[i]) ) {
-			if ( !filter || filter( elem, context, xml ) ) {
-				newUnmatched.push( elem );
-				if ( mapped ) {
-					map.push( i );
-				}
-			}
-		}
-	}
-
-	return newUnmatched;
-}
-
-function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
-	if ( postFilter && !postFilter[ expando ] ) {
-		postFilter = setMatcher( postFilter );
-	}
-	if ( postFinder && !postFinder[ expando ] ) {
-		postFinder = setMatcher( postFinder, postSelector );
-	}
-	return markFunction(function( seed, results, context, xml ) {
-		var temp, i, elem,
-			preMap = [],
-			postMap = [],
-			preexisting = results.length,
-
-			// Get initial elements from seed or context
-			elems = seed || multipleContexts( selector || "*", context.nodeType ? [ context ] : context, [] ),
-
-			// Prefilter to get matcher input, preserving a map for seed-results synchronization
-			matcherIn = preFilter && ( seed || !selector ) ?
-				condense( elems, preMap, preFilter, context, xml ) :
-				elems,
-
-			matcherOut = matcher ?
-				// If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
-				postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
-
-					// ...intermediate processing is necessary
-					[] :
-
-					// ...otherwise use results directly
-					results :
-				matcherIn;
-
-		// Find primary matches
-		if ( matcher ) {
-			matcher( matcherIn, matcherOut, context, xml );
-		}
-
-		// Apply postFilter
-		if ( postFilter ) {
-			temp = condense( matcherOut, postMap );
-			postFilter( temp, [], context, xml );
-
-			// Un-match failing elements by moving them back to matcherIn
-			i = temp.length;
-			while ( i-- ) {
-				if ( (elem = temp[i]) ) {
-					matcherOut[ postMap[i] ] = !(matcherIn[ postMap[i] ] = elem);
-				}
-			}
-		}
-
-		if ( seed ) {
-			if ( postFinder || preFilter ) {
-				if ( postFinder ) {
-					// Get the final matcherOut by condensing this intermediate into postFinder contexts
-					temp = [];
-					i = matcherOut.length;
-					while ( i-- ) {
-						if ( (elem = matcherOut[i]) ) {
-							// Restore matcherIn since elem is not yet a final match
-							temp.push( (matcherIn[i] = elem) );
-						}
-					}
-					postFinder( null, (matcherOut = []), temp, xml );
-				}
-
-				// Move matched elements from seed to results to keep them synchronized
-				i = matcherOut.length;
-				while ( i-- ) {
-					if ( (elem = matcherOut[i]) &&
-						(temp = postFinder ? indexOf( seed, elem ) : preMap[i]) > -1 ) {
-
-						seed[temp] = !(results[temp] = elem);
-					}
-				}
-			}
-
-		// Add elements to results, through postFinder if defined
-		} else {
-			matcherOut = condense(
-				matcherOut === results ?
-					matcherOut.splice( preexisting, matcherOut.length ) :
-					matcherOut
-			);
-			if ( postFinder ) {
-				postFinder( null, results, matcherOut, xml );
-			} else {
-				push.apply( results, matcherOut );
-			}
-		}
-	});
-}
-
-function matcherFromTokens( tokens ) {
-	var checkContext, matcher, j,
-		len = tokens.length,
-		leadingRelative = Expr.relative[ tokens[0].type ],
-		implicitRelative = leadingRelative || Expr.relative[" "],
-		i = leadingRelative ? 1 : 0,
-
-		// The foundational matcher ensures that elements are reachable from top-level context(s)
-		matchContext = addCombinator( function( elem ) {
-			return elem === checkContext;
-		}, implicitRelative, true ),
-		matchAnyContext = addCombinator( function( elem ) {
-			return indexOf( checkContext, elem ) > -1;
-		}, implicitRelative, true ),
-		matchers = [ function( elem, context, xml ) {
-			var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
-				(checkContext = context).nodeType ?
-					matchContext( elem, context, xml ) :
-					matchAnyContext( elem, context, xml ) );
-			// Avoid hanging onto element (issue #299)
-			checkContext = null;
-			return ret;
-		} ];
-
-	for ( ; i < len; i++ ) {
-		if ( (matcher = Expr.relative[ tokens[i].type ]) ) {
-			matchers = [ addCombinator(elementMatcher( matchers ), matcher) ];
-		} else {
-			matcher = Expr.filter[ tokens[i].type ].apply( null, tokens[i].matches );
-
-			// Return special upon seeing a positional matcher
-			if ( matcher[ expando ] ) {
-				// Find the next relative operator (if any) for proper handling
-				j = ++i;
-				for ( ; j < len; j++ ) {
-					if ( Expr.relative[ tokens[j].type ] ) {
-						break;
-					}
-				}
-				return setMatcher(
-					i > 1 && elementMatcher( matchers ),
-					i > 1 && toSelector(
-						// If the preceding token was a descendant combinator, insert an implicit any-element `*`
-						tokens.slice( 0, i - 1 ).concat({ value: tokens[ i - 2 ].type === " " ? "*" : "" })
-					).replace( rtrim, "$1" ),
-					matcher,
-					i < j && matcherFromTokens( tokens.slice( i, j ) ),
-					j < len && matcherFromTokens( (tokens = tokens.slice( j )) ),
-					j < len && toSelector( tokens )
-				);
-			}
-			matchers.push( matcher );
-		}
-	}
-
-	return elementMatcher( matchers );
-}
-
-function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
-	var bySet = setMatchers.length > 0,
-		byElement = elementMatchers.length > 0,
-		superMatcher = function( seed, context, xml, results, outermost ) {
-			var elem, j, matcher,
-				matchedCount = 0,
-				i = "0",
-				unmatched = seed && [],
-				setMatched = [],
-				contextBackup = outermostContext,
-				// We must always have either seed elements or outermost context
-				elems = seed || byElement && Expr.find["TAG"]( "*", outermost ),
-				// Use integer dirruns iff this is the outermost matcher
-				dirrunsUnique = (dirruns += contextBackup == null ? 1 : Math.random() || 0.1),
-				len = elems.length;
-
-			if ( outermost ) {
-				outermostContext = context !== document && context;
-			}
-
-			// Add elements passing elementMatchers directly to results
-			// Keep `i` a string if there are no elements so `matchedCount` will be "00" below
-			// Support: IE<9, Safari
-			// Tolerate NodeList properties (IE: "length"; Safari: <number>) matching elements by id
-			for ( ; i !== len && (elem = elems[i]) != null; i++ ) {
-				if ( byElement && elem ) {
-					j = 0;
-					while ( (matcher = elementMatchers[j++]) ) {
-						if ( matcher( elem, context, xml ) ) {
-							results.push( elem );
-							break;
-						}
-					}
-					if ( outermost ) {
-						dirruns = dirrunsUnique;
-					}
-				}
-
-				// Track unmatched elements for set filters
-				if ( bySet ) {
-					// They will have gone through all possible matchers
-					if ( (elem = !matcher && elem) ) {
-						matchedCount--;
-					}
-
-					// Lengthen the array for every element, matched or not
-					if ( seed ) {
-						unmatched.push( elem );
-					}
-				}
-			}
-
-			// Apply set filters to unmatched elements
-			matchedCount += i;
-			if ( bySet && i !== matchedCount ) {
-				j = 0;
-				while ( (matcher = setMatchers[j++]) ) {
-					matcher( unmatched, setMatched, context, xml );
-				}
-
-				if ( seed ) {
-					// Reintegrate element matches to eliminate the need for sorting
-					if ( matchedCount > 0 ) {
-						while ( i-- ) {
-							if ( !(unmatched[i] || setMatched[i]) ) {
-								setMatched[i] = pop.call( results );
-							}
-						}
-					}
-
-					// Discard index placeholder values to get only actual matches
-					setMatched = condense( setMatched );
-				}
-
-				// Add matches to results
-				push.apply( results, setMatched );
-
-				// Seedless set matches succeeding multiple successful matchers stipulate sorting
-				if ( outermost && !seed && setMatched.length > 0 &&
-					( matchedCount + setMatchers.length ) > 1 ) {
-
-					Sizzle.uniqueSort( results );
-				}
-			}
-
-			// Override manipulation of globals by nested matchers
-			if ( outermost ) {
-				dirruns = dirrunsUnique;
-				outermostContext = contextBackup;
-			}
-
-			return unmatched;
-		};
-
-	return bySet ?
-		markFunction( superMatcher ) :
-		superMatcher;
-}
-
-compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
-	var i,
-		setMatchers = [],
-		elementMatchers = [],
-		cached = compilerCache[ selector + " " ];
-
-	if ( !cached ) {
-		// Generate a function of recursive functions that can be used to check each element
-		if ( !match ) {
-			match = tokenize( selector );
-		}
-		i = match.length;
-		while ( i-- ) {
-			cached = matcherFromTokens( match[i] );
-			if ( cached[ expando ] ) {
-				setMatchers.push( cached );
-			} else {
-				elementMatchers.push( cached );
-			}
-		}
-
-		// Cache the compiled function
-		cached = compilerCache( selector, matcherFromGroupMatchers( elementMatchers, setMatchers ) );
-
-		// Save selector and tokenization
-		cached.selector = selector;
-	}
-	return cached;
-};
-
-/**
- * A low-level selection function that works with Sizzle's compiled
- *  selector functions
- * @param {String|Function} selector A selector or a pre-compiled
- *  selector function built with Sizzle.compile
- * @param {Element} context
- * @param {Array} [results]
- * @param {Array} [seed] A set of elements to match against
- */
-select = Sizzle.select = function( selector, context, results, seed ) {
-	var i, tokens, token, type, find,
-		compiled = typeof selector === "function" && selector,
-		match = !seed && tokenize( (selector = compiled.selector || selector) );
-
-	results = results || [];
-
-	// Try to minimize operations if there is no seed and only one group
-	if ( match.length === 1 ) {
-
-		// Take a shortcut and set the context if the root selector is an ID
-		tokens = match[0] = match[0].slice( 0 );
-		if ( tokens.length > 2 && (token = tokens[0]).type === "ID" &&
-				support.getById && context.nodeType === 9 && documentIsHTML &&
-				Expr.relative[ tokens[1].type ] ) {
-
-			context = ( Expr.find["ID"]( token.matches[0].replace(runescape, funescape), context ) || [] )[0];
-			if ( !context ) {
-				return results;
-
-			// Precompiled matchers will still verify ancestry, so step up a level
-			} else if ( compiled ) {
-				context = context.parentNode;
-			}
-
-			selector = selector.slice( tokens.shift().value.length );
-		}
-
-		// Fetch a seed set for right-to-left matching
-		i = matchExpr["needsContext"].test( selector ) ? 0 : tokens.length;
-		while ( i-- ) {
-			token = tokens[i];
-
-			// Abort if we hit a combinator
-			if ( Expr.relative[ (type = token.type) ] ) {
-				break;
-			}
-			if ( (find = Expr.find[ type ]) ) {
-				// Search, expanding context for leading sibling combinators
-				if ( (seed = find(
-					token.matches[0].replace( runescape, funescape ),
-					rsibling.test( tokens[0].type ) && testContext( context.parentNode ) || context
-				)) ) {
-
-					// If seed is empty or no tokens remain, we can return early
-					tokens.splice( i, 1 );
-					selector = seed.length && toSelector( tokens );
-					if ( !selector ) {
-						push.apply( results, seed );
-						return results;
-					}
-
-					break;
-				}
-			}
-		}
-	}
-
-	// Compile and execute a filtering function if one is not provided
-	// Provide `match` to avoid retokenization if we modified the selector above
-	( compiled || compile( selector, match ) )(
-		seed,
-		context,
-		!documentIsHTML,
-		results,
-		rsibling.test( selector ) && testContext( context.parentNode ) || context
-	);
-	return results;
-};
-
-// One-time assignments
-
-// Sort stability
-support.sortStable = expando.split("").sort( sortOrder ).join("") === expando;
-
-// Support: Chrome 14-35+
-// Always assume duplicates if they aren't passed to the comparison function
-support.detectDuplicates = !!hasDuplicate;
-
-// Initialize against the default document
-setDocument();
-
-// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27)
-// Detached nodes confoundingly follow *each other*
-support.sortDetached = assert(function( div1 ) {
-	// Should return 1, but returns 4 (following)
-	return div1.compareDocumentPosition( document.createElement("div") ) & 1;
-});
-
-// Support: IE<8
-// Prevent attribute/property "interpolation"
-// http://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
-if ( !assert(function( div ) {
-	div.innerHTML = "<a href='#'></a>";
-	return div.firstChild.getAttribute("href") === "#" ;
-}) ) {
-	addHandle( "type|href|height|width", function( elem, name, isXML ) {
-		if ( !isXML ) {
-			return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
-		}
-	});
-}
-
-// Support: IE<9
-// Use defaultValue in place of getAttribute("value")
-if ( !support.attributes || !assert(function( div ) {
-	div.innerHTML = "<input/>";
-	div.firstChild.setAttribute( "value", "" );
-	return div.firstChild.getAttribute( "value" ) === "";
-}) ) {
-	addHandle( "value", function( elem, name, isXML ) {
-		if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
-			return elem.defaultValue;
-		}
-	});
-}
-
-// Support: IE<9
-// Use getAttributeNode to fetch booleans when getAttribute lies
-if ( !assert(function( div ) {
-	return div.getAttribute("disabled") == null;
-}) ) {
-	addHandle( booleans, function( elem, name, isXML ) {
-		var val;
-		if ( !isXML ) {
-			return elem[ name ] === true ? name.toLowerCase() :
-					(val = elem.getAttributeNode( name )) && val.specified ?
-					val.value :
-				null;
-		}
-	});
-}
-
-return Sizzle;
-
-})( window );
-
-
-
-jQuery.find = Sizzle;
-jQuery.expr = Sizzle.selectors;
-jQuery.expr[":"] = jQuery.expr.pseudos;
-jQuery.unique = Sizzle.uniqueSort;
-jQuery.text = Sizzle.getText;
-jQuery.isXMLDoc = Sizzle.isXML;
-jQuery.contains = Sizzle.contains;
-
-
-
-var rneedsContext = jQuery.expr.match.needsContext;
-
-var rsingleTag = (/^<(\w+)\s*\/?>(?:<\/\1>|)$/);
-
-
-
-var risSimple = /^.[^:#\[\.,]*$/;
-
-// Implement the identical functionality for filter and not
-function winnow( elements, qualifier, not ) {
-	if ( jQuery.isFunction( qualifier ) ) {
-		return jQuery.grep( elements, function( elem, i ) {
-			/* jshint -W018 */
-			return !!qualifier.call( elem, i, elem ) !== not;
-		});
-
-	}
-
-	if ( qualifier.nodeType ) {
-		return jQuery.grep( elements, function( elem ) {
-			return ( elem === qualifier ) !== not;
-		});
-
-	}
-
-	if ( typeof qualifier === "string" ) {
-		if ( risSimple.test( qualifier ) ) {
-			return jQuery.filter( qualifier, elements, not );
-		}
-
-		qualifier = jQuery.filter( qualifier, elements );
-	}
-
-	return jQuery.grep( elements, function( elem ) {
-		return ( indexOf.call( qualifier, elem ) >= 0 ) !== not;
-	});
-}
-
-jQuery.filter = function( expr, elems, not ) {
-	var elem = elems[ 0 ];
-
-	if ( not ) {
-		expr = ":not(" + expr + ")";
-	}
-
-	return elems.length === 1 && elem.nodeType === 1 ?
-		jQuery.find.matchesSelector( elem, expr ) ? [ elem ] : [] :
-		jQuery.find.matches( expr, jQuery.grep( elems, function( elem ) {
-			return elem.nodeType === 1;
-		}));
-};
-
-jQuery.fn.extend({
-	find: function( selector ) {
-		var i,
-			len = this.length,
-			ret = [],
-			self = this;
-
-		if ( typeof selector !== "string" ) {
-			return this.pushStack( jQuery( selector ).filter(function() {
-				for ( i = 0; i < len; i++ ) {
-					if ( jQuery.contains( self[ i ], this ) ) {
-						return true;
-					}
-				}
-			}) );
-		}
-
-		for ( i = 0; i < len; i++ ) {
-			jQuery.find( selector, self[ i ], ret );
-		}
-
-		// Needed because $( selector, context ) becomes $( context ).find( selector )
-		ret = this.pushStack( len > 1 ? jQuery.unique( ret ) : ret );
-		ret.selector = this.selector ? this.selector + " " + selector : selector;
-		return ret;
-	},
-	filter: function( selector ) {
-		return this.pushStack( winnow(this, selector || [], false) );
-	},
-	not: function( selector ) {
-		return this.pushStack( winnow(this, selector || [], true) );
-	},
-	is: function( selector ) {
-		return !!winnow(
-			this,
-
-			// If this is a positional/relative selector, check membership in the returned set
-			// so $("p:first").is("p:last") won't return true for a doc with two "p".
-			typeof selector === "string" && rneedsContext.test( selector ) ?
-				jQuery( selector ) :
-				selector || [],
-			false
-		).length;
-	}
-});
-
-
-// Initialize a jQuery object
-
-
-// A central reference to the root jQuery(document)
-var rootjQuery,
-
-	// A simple way to check for HTML strings
-	// Prioritize #id over <tag> to avoid XSS via location.hash (#9521)
-	// Strict HTML recognition (#11290: must start with <)
-	rquickExpr = /^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,
-
-	init = jQuery.fn.init = function( selector, context ) {
-		var match, elem;
-
-		// HANDLE: $(""), $(null), $(undefined), $(false)
-		if ( !selector ) {
-			return this;
-		}
-
-		// Handle HTML strings
-		if ( typeof selector === "string" ) {
-			if ( selector[0] === "<" && selector[ selector.length - 1 ] === ">" && selector.length >= 3 ) {
-				// Assume that strings that start and end with <> are HTML and skip the regex check
-				match = [ null, selector, null ];
-
-			} else {
-				match = rquickExpr.exec( selector );
-			}
-
-			// Match html or make sure no context is specified for #id
-			if ( match && (match[1] || !context) ) {
-
-				// HANDLE: $(html) -> $(array)
-				if ( match[1] ) {
-					context = context instanceof jQuery ? context[0] : context;
-
-					// Option to run scripts is true for back-compat
-					// Intentionally let the error be thrown if parseHTML is not present
-					jQuery.merge( this, jQuery.parseHTML(
-						match[1],
-						context && context.nodeType ? context.ownerDocument || context : document,
-						true
-					) );
-
-					// HANDLE: $(html, props)
-					if ( rsingleTag.test( match[1] ) && jQuery.isPlainObject( context ) ) {
-						for ( match in context ) {
-							// Properties of context are called as methods if possible
-							if ( jQuery.isFunction( this[ match ] ) ) {
-								this[ match ]( context[ match ] );
-
-							// ...and otherwise set as attributes
-							} else {
-								this.attr( match, context[ match ] );
-							}
-						}
-					}
-
-					return this;
-
-				// HANDLE: $(#id)
-				} else {
-					elem = document.getElementById( match[2] );
-
-					// Support: Blackberry 4.6
-					// gEBID returns nodes no longer in the document (#6963)
-					if ( elem && elem.parentNode ) {
-						// Inject the element directly into the jQuery object
-						this.length = 1;
-						this[0] = elem;
-					}
-
-					this.context = document;
-					this.selector = selector;
-					return this;
-				}
-
-			// HANDLE: $(expr, $(...))
-			} else if ( !context || context.jquery ) {
-				return ( context || rootjQuery ).find( selector );
-
-			// HANDLE: $(expr, context)
-			// (which is just equivalent to: $(context).find(expr)
-			} else {
-				return this.constructor( context ).find( selector );
-			}
-
-		// HANDLE: $(DOMElement)
-		} else if ( selector.nodeType ) {
-			this.context = this[0] = selector;
-			this.length = 1;
-			return this;
-
-		// HANDLE: $(function)
-		// Shortcut for document ready
-		} else if ( jQuery.isFunction( selector ) ) {
-			return typeof rootjQuery.ready !== "undefined" ?
-				rootjQuery.ready( selector ) :
-				// Execute immediately if ready is not present
-				selector( jQuery );
-		}
-
-		if ( selector.selector !== undefined ) {
-			this.selector = selector.selector;
-			this.context = selector.context;
-		}
-
-		return jQuery.makeArray( selector, this );
-	};
-
-// Give the init function the jQuery prototype for later instantiation
-init.prototype = jQuery.fn;
-
-// Initialize central reference
-rootjQuery = jQuery( document );
-
-
-var rparentsprev = /^(?:parents|prev(?:Until|All))/,
-	// Methods guaranteed to produce a unique set when starting from a unique set
-	guaranteedUnique = {
-		children: true,
-		contents: true,
-		next: true,
-		prev: true
-	};
-
-jQuery.extend({
-	dir: function( elem, dir, until ) {
-		var matched = [],
-			truncate = until !== undefined;
-
-		while ( (elem = elem[ dir ]) && elem.nodeType !== 9 ) {
-			if ( elem.nodeType === 1 ) {
-				if ( truncate && jQuery( elem ).is( until ) ) {
-					break;
-				}
-				matched.push( elem );
-			}
-		}
-		return matched;
-	},
-
-	sibling: function( n, elem ) {
-		var matched = [];
-
-		for ( ; n; n = n.nextSibling ) {
-			if ( n.nodeType === 1 && n !== elem ) {
-				matched.push( n );
-			}
-		}
-
-		return matched;
-	}
-});
-
-jQuery.fn.extend({
-	has: function( target ) {
-		var targets = jQuery( target, this ),
-			l = targets.length;
-
-		return this.filter(function() {
-			var i = 0;
-			for ( ; i < l; i++ ) {
-				if ( jQuery.contains( this, targets[i] ) ) {
-					return true;
-				}
-			}
-		});
-	},
-
-	closest: function( selectors, context ) {
-		var cur,
-			i = 0,
-			l = this.length,
-			matched = [],
-			pos = rneedsContext.test( selectors ) || typeof selectors !== "string" ?
-				jQuery( selectors, context || this.context ) :
-				0;
-
-		for ( ; i < l; i++ ) {
-			for ( cur = this[i]; cur && cur !== context; cur = cur.parentNode ) {
-				// Always skip document fragments
-				if ( cur.nodeType < 11 && (pos ?
-					pos.index(cur) > -1 :
-
-					// Don't pass non-elements to Sizzle
-					cur.nodeType === 1 &&
-						jQuery.find.matchesSelector(cur, selectors)) ) {
-
-					matched.push( cur );
-					break;
-				}
-			}
-		}
-
-		return this.pushStack( matched.length > 1 ? jQuery.unique( matched ) : matched );
-	},
-
-	// Determine the position of an element within the set
-	index: function( elem ) {
-
-		// No argument, return index in parent
-		if ( !elem ) {
-			return ( this[ 0 ] && this[ 0 ].parentNode ) ? this.first().prevAll().length : -1;
-		}
-
-		// Index in selector
-		if ( typeof elem === "string" ) {
-			return indexOf.call( jQuery( elem ), this[ 0 ] );
-		}
-
-		// Locate the position of the desired element
-		return indexOf.call( this,
-
-			// If it receives a jQuery object, the first element is used
-			elem.jquery ? elem[ 0 ] : elem
-		);
-	},
-
-	add: function( selector, context ) {
-		return this.pushStack(
-			jQuery.unique(
-				jQuery.merge( this.get(), jQuery( selector, context ) )
-			)
-		);
-	},
-
-	addBack: function( selector ) {
-		return this.add( selector == null ?
-			this.prevObject : this.prevObject.filter(selector)
-		);
-	}
-});
-
-function sibling( cur, dir ) {
-	while ( (cur = cur[dir]) && cur.nodeType !== 1 ) {}
-	return cur;
-}
-
-jQuery.each({
-	parent: function( elem ) {
-		var parent = elem.parentNode;
-		return parent && parent.nodeType !== 11 ? parent : null;
-	},
-	parents: function( elem ) {
-		return jQuery.dir( elem, "parentNode" );
-	},
-	parentsUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "parentNode", until );
-	},
-	next: function( elem ) {
-		return sibling( elem, "nextSibling" );
-	},
-	prev: function( elem ) {
-		return sibling( elem, "previousSibling" );
-	},
-	nextAll: function( elem ) {
-		return jQuery.dir( elem, "nextSibling" );
-	},
-	prevAll: function( elem ) {
-		return jQuery.dir( elem, "previousSibling" );
-	},
-	nextUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "nextSibling", until );
-	},
-	prevUntil: function( elem, i, until ) {
-		return jQuery.dir( elem, "previousSibling", until );
-	},
-	siblings: function( elem ) {
-		return jQuery.sibling( ( elem.parentNode || {} ).firstChild, elem );
-	},
-	children: function( elem ) {
-		return jQuery.sibling( elem.firstChild );
-	},
-	contents: function( elem ) {
-		return elem.contentDocument || jQuery.merge( [], elem.childNodes );
-	}
-}, function( name, fn ) {
-	jQuery.fn[ name ] = function( until, selector ) {
-		var matched = jQuery.map( this, fn, until );
-
-		if ( name.slice( -5 ) !== "Until" ) {
-			selector = until;
-		}
-
-		if ( selector && typeof selector === "string" ) {
-			matched = jQuery.filter( selector, matched );
-		}
-
-		if ( this.length > 1 ) {
-			// Remove duplicates
-			if ( !guaranteedUnique[ name ] ) {
-				jQuery.unique( matched );
-			}
-
-			// Reverse order for parents* and prev-derivatives
-			if ( rparentsprev.test( name ) ) {
-				matched.reverse();
-			}
-		}
-
-		return this.pushStack( matched );
-	};
-});
-var rnotwhite = (/\S+/g);
-
-
-
-// String to Object options format cache
-var optionsCache = {};
-
-// Convert String-formatted options into Object-formatted ones and store in cache
-function createOptions( options ) {
-	var object = optionsCache[ options ] = {};
-	jQuery.each( options.match( rnotwhite ) || [], function( _, flag ) {
-		object[ flag ] = true;
-	});
-	return object;
-}
-
-/*
- * Create a callback list using the following parameters:
- *
- *	options: an optional list of space-separated options that will change how
- *			the callback list behaves or a more traditional option object
- *
- * By default a callback list will act like an event callback list and can be
- * "fired" multiple times.
- *
- * Possible options:
- *
- *	once:			will ensure the callback list can only be fired once (like a Deferred)
- *
- *	memory:			will keep track of previous values and will call any callback added
- *					after the list has been fired right away with the latest "memorized"
- *					values (like a Deferred)
- *
- *	unique:			will ensure a callback can only be added once (no duplicate in the list)
- *
- *	stopOnFalse:	interrupt callings when a callback returns false
- *
- */
-jQuery.Callbacks = function( options ) {
-
-	// Convert options from String-formatted to Object-formatted if needed
-	// (we check in cache first)
-	options = typeof options === "string" ?
-		( optionsCache[ options ] || createOptions( options ) ) :
-		jQuery.extend( {}, options );
-
-	var // Last fire value (for non-forgettable lists)
-		memory,
-		// Flag to know if list was already fired
-		fired,
-		// Flag to know if list is currently firing
-		firing,
-		// First callback to fire (used internally by add and fireWith)
-		firingStart,
-		// End of the loop when firing
-		firingLength,
-		// Index of currently firing callback (modified by remove if needed)
-		firingIndex,
-		// Actual callback list
-		list = [],
-		// Stack of fire calls for repeatable lists
-		stack = !options.once && [],
-		// Fire callbacks
-		fire = function( data ) {
-			memory = options.memory && data;
-			fired = true;
-			firingIndex = firingStart || 0;
-			firingStart = 0;
-			firingLength = list.length;
-			firing = true;
-			for ( ; list && firingIndex < firingLength; firingIndex++ ) {
-				if ( list[ firingIndex ].apply( data[ 0 ], data[ 1 ] ) === false && options.stopOnFalse ) {
-					memory = false; // To prevent further calls using add
-					break;
-				}
-			}
-			firing = false;
-			if ( list ) {
-				if ( stack ) {
-					if ( stack.length ) {
-						fire( stack.shift() );
-					}
-				} else if ( memory ) {
-					list = [];
-				} else {
-					self.disable();
-				}
-			}
-		},
-		// Actual Callbacks object
-		self = {
-			// Add a callback or a collection of callbacks to the list
-			add: function() {
-				if ( list ) {
-					// First, we save the current length
-					var start = list.length;
-					(function add( args ) {
-						jQuery.each( args, function( _, arg ) {
-							var type = jQuery.type( arg );
-							if ( type === "function" ) {
-								if ( !options.unique || !self.has( arg ) ) {
-									list.push( arg );
-								}
-							} else if ( arg && arg.length && type !== "string" ) {
-								// Inspect recursively
-								add( arg );
-							}
-						});
-					})( arguments );
-					// Do we need to add the callbacks to the
-					// current firing batch?
-					if ( firing ) {
-						firingLength = list.length;
-					// 

<TRUNCATED>