You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2017/01/20 05:39:22 UTC

[01/17] flink git commit: [FLINK-5380] Fix task metrics reuse for single-operator chains

Repository: flink
Updated Branches:
  refs/heads/release-1.2 92e393500 -> 080617d28


[FLINK-5380] Fix task metrics reuse for single-operator chains


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

Branch: refs/heads/release-1.2
Commit: 792f7e45216377fa1d6f29dfc767d83cf1a84f37
Parents: 28c18e2
Author: zentol <ch...@apache.org>
Authored: Thu Jan 5 14:37:03 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:41:59 2017 +0100

----------------------------------------------------------------------
 .../api/graph/StreamingJobGraphGenerator.java   |  6 ++--
 .../graph/StreamingJobGraphGeneratorTest.java   | 38 ++++++++++++++++++++
 2 files changed, 41 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/792f7e45/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index 1bfaf3f..f562b98 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -234,9 +234,9 @@ public class StreamingJobGraphGenerator {
 				config.setChainIndex(chainIndex);
 				config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName());
 				chainedConfigs.get(startNodeId).put(currentNodeId, config);
-				if (chainableOutputs.isEmpty()) {
-					config.setChainEnd();
-				}
+			}
+			if (chainableOutputs.isEmpty()) {
+				config.setChainEnd();
 			}
 
 			return transitiveOutEdges;

http://git-wip-us.apache.org/repos/asf/flink/blob/792f7e45/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
index b817c93..4d462d0 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
@@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -31,11 +32,13 @@ import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Map;
 import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 @SuppressWarnings("serial")
 public class StreamingJobGraphGeneratorTest extends TestLogger {
@@ -170,4 +173,39 @@ public class StreamingJobGraphGeneratorTest extends TestLogger {
 		JobSnapshottingSettings snapshottingSettings = jobGraph.getSnapshotSettings();
 		assertEquals(Long.MAX_VALUE, snapshottingSettings.getCheckpointInterval());
 	}
+
+	/**
+	 * Verifies that the chain start/end is correctly set.
+	 */
+	@Test
+	public void testChainStartEndSetting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		// fromElements -> CHAIN(Map -> Print)
+		env.fromElements(1, 2, 3)
+			.map(new MapFunction<Integer, Integer>() {
+				@Override
+				public Integer map(Integer value) throws Exception {
+					return value;
+				}
+			})
+			.print();
+		JobGraph jobGraph = new StreamingJobGraphGenerator(env.getStreamGraph()).createJobGraph();
+
+		JobVertex sourceVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(0);
+		JobVertex mapPrintVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(1);
+
+		StreamConfig sourceConfig = new StreamConfig(sourceVertex.getConfiguration());
+		StreamConfig mapConfig = new StreamConfig(mapPrintVertex.getConfiguration());
+		Map<Integer, StreamConfig> chainedConfigs = mapConfig.getTransitiveChainedTaskConfigs(getClass().getClassLoader());
+		StreamConfig printConfig = chainedConfigs.get(3);
+
+		assertTrue(sourceConfig.isChainStart());
+		assertTrue(sourceConfig.isChainEnd());
+
+		assertTrue(mapConfig.isChainStart());
+		assertFalse(mapConfig.isChainEnd());
+
+		assertFalse(printConfig.isChainStart());
+		assertTrue(printConfig.isChainEnd());
+	}
 }


[08/17] flink git commit: [FLINK-5417] [docs] Fix config file name in slots_parallelism.svg

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/7f20bd0d/docs/setup/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/setup/yarn_setup.md b/docs/setup/yarn_setup.md
index 5f6855f..53423b8 100644
--- a/docs/setup/yarn_setup.md
+++ b/docs/setup/yarn_setup.md
@@ -118,7 +118,7 @@ Please note that the Client requires the `YARN_CONF_DIR` or `HADOOP_CONF_DIR` en
 ./bin/yarn-session.sh -n 10 -tm 8192 -s 32
 ~~~
 
-The system will use the configuration in `conf/flink-config.yaml`. Please follow our [configuration guide](config.html) if you want to change something.
+The system will use the configuration in `conf/flink-conf.yaml`. Please follow our [configuration guide](config.html) if you want to change something.
 
 Flink on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines), `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN) and `parallelism.default` if the number of slots has been specified.
 


[16/17] flink git commit: Rebuild web-dashboard

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/080617d2/flink-runtime-web/web-dashboard/web/js/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/index.js b/flink-runtime-web/web-dashboard/web/js/index.js
index e921e6b..93d395e 100644
--- a/flink-runtime-web/web-dashboard/web/js/index.js
+++ b/flink-runtime-web/web-dashboard/web/js/index.js
@@ -1,2 +1,2 @@
-angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,o){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("overview",{url:"/overview",views:{main:{templateUrl:"partials/overview.html",controller:"OverviewController"}}}).state("running-jobs",{url:"/running-jobs",views:{main:{templateUrl:"parti
 als/jobs/running-jobs.html",controller:"RunningJobsController"}}}).state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanage
 rs.html",controller:"JobPlanTaskManagersController"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partia
 ls/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-
 job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}).state("all-manager",{url:"/taskmanagers",views:{main:{templateUrl:"partials/taskmanager/index.html",controller:"AllTaskManagersController"}}}).state("single-manager",{url:"/taskmanager/{taskmanagerid}","abstract":!0,views:{main:{templateUrl:"partials/taskmanager/taskmanager.html",controller:"SingleTaskManagerController"}}}).state("single-manager.metrics",{url:"/metrics",views:{details:{templateUrl:"partials/taskmanager/taskmanager.metrics.html"}}}).state("single-manager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/taskmanager/taskmanager.stdout.html",controller:"SingleTaskManagerStdoutController"}}}).state("single-manager.log",{url:"/log",views:{details:{templateUrl:"partials/taskmanager/taskmanager.log.html",controller:"Singl
 eTaskManagerLogsController"}}}).state("jobmanager",{url:"/jobmanager",views:{main:{templateUrl:"partials/jobmanager/index.html"}}}).state("jobmanager.config",{url:"/config",views:{details:{templateUrl:"partials/jobmanager/config.html",controller:"JobManagerConfigController"}}}).state("jobmanager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/jobmanager/stdout.html",controller:"JobManagerStdoutController"}}}).state("jobmanager.log",{url:"/log",views:{details:{templateUrl:"partials/jobmanager/log.html",controller:"JobManagerLogsController"}}}).state("submit",{url:"/submit",views:{main:{templateUrl:"partials/submit.html",controller:"JobSubmitController"}}}),t.otherwise("/overview")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLabelClass=fun
 ction(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e)
 {var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,s,a;return"undefined"==typeof e||null===e?"":(i=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),o=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===o?0===s?i+"ms":s+"s ":o+"m "+s+"s":t?n+"h "+o+"m":n+"h "+o+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+s+"s")}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),e<o?(e/o).toFixed(2)+" "+r[n]:e<1e3*o?(e/o).toPrecision(3)+" "+r[n]:t(e,n+1)},"undefined"==typeof e||null===e?"":e<1e3?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filt
 er("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"config").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){return t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadData=function(){return t.loadLogs().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(
 e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,o){return o=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadStdout=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},
 this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval",function(e,t,r,n,o,i,s,a){var l;return e.jobid=r.jobid,e.job=null,e.plan=null,e.vertices=null,e.backPressureOperatorStats={},n.loadJob(r.jobid).then(function(t){return e.job=t,e.plan=t.plan,e.vertices=t.vertices,o.setupMetrics(r.job
 id,t.vertices)}),l=a(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},s["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.vertices=null,e.backPressureOperatorStats=null,a.cancel(l)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})}}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.
 nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanTaskManagersController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccu
 mulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return 
 e.subtaskDetails[r]=t})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(o(r.checkpointId),e.nodeid)return i(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$sco
 pe","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),n=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t,e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},e.dropped=function(t,o,i,s,a){return r.orderMetrics(e.jobid,e.nodeid,i,o),e.$broadcast("metrics:refresh",i),n(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodei
 d,t.id),n()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),n()},e.setMetricSize=function(t,o){return r.setMetricSize(e.jobid,e.nodeid,t,o),n()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return n()}),e.nodeid)return n()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.F
 INISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,s,a;s=r.children()[0],i=r.width(),angular.element(s).attr("width",i),a=function(e){return e.replace("&gt;",">")},o=function(r){var n,o,i;return d3.select(s).selectAll("*").remove(),i=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?i.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_t
 ime:e["end-time"],type:e.type}]}):i.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,o){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(s).datum(i).call(n)},t.$watch(n.vertices,function(e){if(e)return o(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph' 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 b
 tn-default zoom-out' ng-click='zoomOut()'><i class='fa fa-minus' /></a> </div>",scope:{plan:"=",setNode:"&"},link:function(e,t,r){var n,o,i,s,a,l,u,c,d,f,p,g,m,b,h,v,k,j,S,C,w,$;p=null,S=d3.behavior.zoom(),$=[],b=r.jobid,k=t.children()[0],v=t.children().children()[0],j=t.children()[1],l=d3.select(k),u=d3.select(v),c=d3.select(j),n=t.width(),angular.element(t.children()[0]).width(n),e.zoomIn=function(){var e,t,r;if(S.scale()<2.99)return e=S.translate(),t=e[0]*(S.scale()+.1/S.scale()),r=e[1]*(S.scale()+.1/S.scale()),S.scale(S.scale()+.1),S.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+S.scale()+")")},e.zoomOut=function(){var e,t,r;if(S.scale()>.31)return S.scale(S.scale()-.1),e=S.translate(),t=e[0]*(S.scale()-.1/S.scale()),r=e[1]*(S.scale()-.1/S.scale()),S.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+S.scale()+")")},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_str
 ategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},m=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},g=function(e,t){return"mirror"===t?"node-mirror":m(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var o,i;return o="<div href='#/jobs/"+b+"/vertex/"+e.id+"' class='node-label "+g(e,t)+"'>",o+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?o+="":(i=e.description,i=w(i),o+="<h4 class='step-name'>"+i+"</h4>"),null!=e.step_function?o+=f(e.id,r,n):(m(t)&&(o+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(o+="<h5>Parallelism: "+e.parallelism+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(o+="<h5>Operation: "+w(e.operator_strategy)+"</h5>")),o+="</div>"},f=function(e,t,r){var n,o;return o="svg-"+e,n="<svg class
 ='"+o+"' width="+t+" height="+r+"><g /></svg>"},w=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e.length>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,o,i){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",o,i),labelType:"html","class":g(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",o,i),labelType:"html","class":g(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",o,i),labelType:"html","class":g(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",o,i),labelType:"html","class":g(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",o,i),labelType:"html","class":g(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",o,i),labelType:"html","class":g(r,"solutionDelta")}):e.setNode(r.id
 ,{label:s(r,"",o,i),labelType:"html","class":g(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},h=function(e,t){var r,n,i,s,l,u,d,f,p,g,m,b,v,k;for(n=[],null!=t.nodes?k=t.nodes:(k=t.step_function,i=!0),s=0,u=k.length;s<u;s++)if(r=k[s],p=0,f=0,r.step_function&&(v=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),$[r.id]=v,h(v,r),m=new dagreD3.render,c.select("g").call(m,v),p=v.graph().width,f=v.graph().height,angular.element(j).empty()),a(e,t,r,i,p,f),n.push(r.id),null!=r.inputs)for(b=r.inputs,l=0,d=b.length;l<d;l++)g=b[l],o(e,t,r,n,g);return e},C=function(e,t){var r,n,o;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)for(o in r.step_function)if(r.step_function[o].id===t)return r.step_function[o]}},d=function(t){var r,n,o,i,s,a;p=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:70,edgesep
 :0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),h(p,t),o=new dagreD3.render,u.call(o,p);for(r in $)i=$[r],l.select("svg.svg-"+r+" g").call(o,i);return n=.5,s=Math.floor((angular.element(k).width()-p.graph().width*n)/2),a=Math.floor((angular.element(k).height()-p.graph().height*n)/2),S.scale(n).translate([s,a]),u.attr("transform","translate("+s+", "+a+") scale("+S.scale()+")"),S.on("zoom",function(){var e;return e=d3.event,u.attr("transform","translate("+e.translate+") scale("+e.scale+")")}),S(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})},e.$watch(r.plan,function(e){if(e)return d(e)})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log","amMoment","$q","$timeout",function(e,t,r,n,o,i){var s,a,l,u,c,d;return s=null,a=null,l={},c={running:[],finished:[],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=
 function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLowerCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){if(!(e["end-time"]>-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"joboverview").success(function(e){return function(t,n,o,i){return
  angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=e.setEndTimes(t)}}),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,a,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),s=o,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,o,i,s;for(n=0,o=t.length;n<o;n++){if(i=t[n],i.id===e)return i;if(i.step_function&&(s=r(e,i.step_function)),s)return s}return null},t=o.defer(),l.job.promise.then(function(n){return function(o){var i;return i=r(e,s.plan.nodes),i.vertex=n.seekVertex(e),t.resolve(i)
 }}(this)),t.promise},this.seekVertex=function(e){var t,r,n,o;for(n=s.vertices,t=0,r=n.length;t<r;t++)if(o=n[t],o.id===e)return o;return null},this.getVertex=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(i){var a;return a=o.seekVertex(r),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return a.subtasks=e.subtasks,n.resolve(a)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r).success(function(e){var t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=o.defer(),l.job.promis
 e.then(function(o){return function(o){return console.log(s.jid),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/accumulators").success(function(o){var i;return i=o["user-accumulators"],e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:i,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints").success(function(e,t,n,o){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){r
 eturn function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var i;return i=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?i.resolve(null):i.resolve(e)})}}(this)),i.promise},this.getOperatorBackPressure=function(r){var n;return n=o.defer(),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/backpressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPressureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"danger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return 
 function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/exceptions").success(function(e){return s.exceptions=e,r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel")},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.size != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {active: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div> <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg /> </div> </div>',replace:!0,scope:{metric:"="
 ,window:"=",removeMetric:"&",setMetricSize:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t<Math.pow(10,n+o)?r=!0:n+=o;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.sh
 owChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),e.showChart(),e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this)
 ,r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,
-this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==localStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(localStorage.flinkMetrics)},this.saveSetup=function(){return localStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestam
 p,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,s){if(i.id===r.id&&(o.metrics[e][t].splice(s,1),s<n))return n-=1}}(this)),this.metrics[e][t].splice(n,0,r),this.
 saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){return _.isString(e)?{id:e,size:"small"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(o,i){var s;return n.setupLSFor(o,i),s=t.defer(),e.get(r.jobServer+"jobs/"+o+"/vertices/"+i+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var s;if(s=n.metrics[o][i].indexOf(e.id),s===-1&&(s=_.findIndex(n.metrics[o][i],{id:e.id})),s===-1)return t.push(e)}),s.resolve(t)}),s.promise}}(this),this.getAllAvailableMetrics=function(n){return function(n,o){var i;return i=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics").success(function(e){return i.resolve(e)}),i.promise}}(this),this.getMetrics=function(n,o,i){var s,a;return s=t.defer(),a=i.join(","),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics?get="+a).success(function(e){return function(t){var r,i;return i={},angular.forEach(t,function(e,t){return i[e.id]=parseInt(e
 .value)}),r={timestamp:Date.now(),values:i},e.saveValue(n,o,r),s.resolve(r)}}(this)),s.promise},this.setupLS(),this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),i=n(function(){return t.loadOverview().then(function(t){return e.overview=t})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadOverview=function(){var o;return o=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.mo
 dule("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","$state","$location",function(e,t,r,n,o,i){var s;return e.yarn=i.absUrl().indexOf("/proxy/application_")!==-1,e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,e.noaccess=t.error,e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","submit-button":"Submit","action-time":0}},e.defaultState(),e.uploader={},e.loadList(),s=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(s)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spi
 nner"),t.deleteJar(n).then(function(e){if(angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error)return alert(e.error)})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r;if("Show Plan"===e.state["plan-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){if(r===e.state["action-time"])return e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan})},e.runJob=function(){var r;if("Submit"===e.state["submit-button"])return r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submitting",e.state["plan-button"]="Show Plan",e.error=null,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.st
 ate.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:e.state.allowNonRestoredState}).then(function(t){if(r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid))return o.go("single-job.plan.subtasks",{jobid:t.jobid})})},e.nodeid=null,e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.$broadcast("reload")):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null)},e.clearFiles=function(){return e.uploader={}},e.uploadFiles=function(t){return e.uploader={},1===t.length?(e.uploader.file=t[0],e.uploader.upload=!0):e.uploader.error="Did ya forget to select a file?"},e.startUpload=function(){var t,r;return null!=e.uploader.file?(t=new FormData,t.append("jarfile",e.uploader.file),e.uploader.upload=!1,e.uploader.success="Initializing upload...",r=new XMLHttpRequest,r.upload.onprogress=function(t){return e.upload
 er.success=null,e.uploader.progress=parseInt(100*t.loaded/t.total)},r.upload.onerror=function(t){return e.uploader.progress=null,e.uploader.error="An error occurred while uploading your file"},r.upload.onload=function(t){return e.uploader.progress=null,e.uploader.success="Saving..."},r.onreadystatechange=function(){var t;if(4===r.readyState)return t=JSON.parse(r.responseText),null!=t.error?(e.uploader.error=t.error,e.uploader.success=null):e.uploader.success="Uploaded!"},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelectClass",function(){return function(e,t){return e===t?"fa-check-square":"fa-square-o"}}),angular.module("flinkApp").service("JobSubmitService",["$http","flinkConfig","$q",function(e,t,r){return this.loadJarList=function(){var n;return n=r.defer(),e.get(t.jobServer+"jars/").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this.deleteJar=function(n){var o;return o=r.defer(),e["del
 ete"](t.jobServer+"jars/"+encodeURIComponent(n)).success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.getPlan=function(n,o){var i;return i=r.defer(),e.get(t.jobServer+"jars/"+encodeURIComponent(n)+"/plan",{params:o}).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.runJob=function(n,o){var i;return i=r.defer(),e.post(t.jobServer+"jars/"+encodeURIComponent(n)+"/run",{},{params:o}).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("AllTaskManagersController",["$scope","TaskManagersService","$interval","flinkConfig",function(e,t,r,n){var o;return t.loadManagers().then(function(t){return e.managers=t}),o=r(function(){return t.loadManagers().then(function(t){return e.managers=t})},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(o)})}]).controller("SingleTaskManagerController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.me
 trics={},r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]}),i=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]).controller("SingleTaskManagerLogsController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.log={},e.taskmanagerid=t.taskmanagerid,r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t}),e.reloadData=function(){return r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t})}}]).controller("SingleTaskManagerStdoutController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.stdout={},e.taskmanagerid=t.taskmanagerid,r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t}),e.reloadData=function(){return r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t})}}]),angular.module("flinkApp").serv
 ice("TaskManagersService",["$http","flinkConfig","$q",function(e,t,r){return this.loadManagers=function(){var n;return n=r.defer(),e.get(t.jobServer+"taskmanagers").success(function(e,t,r,o){return n.resolve(e.taskmanagers)}),n.promise},this}]).service("SingleTaskManagerService",["$http","flinkConfig","$q",function(e,t,r){return this.loadMetrics=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/metrics").success(function(e,t,r,n){return o.resolve(e.taskmanagers)}),o.promise},this.loadLogs=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/log").success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.loadStdout=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/stdout").success(function(e,t,r,n){return o.resolve(e)}),o.promise},this}]);
\ No newline at end of file
+angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,o){return r.redirectTo?(e.preventDefault(),t.go(r.redirectTo,n)):void 0})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.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":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.task
 managers.html",controller:"JobPlanTaskManagersController"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"
 partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("s
 ingle-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}).state("all-manager",{url:"/taskmanagers",views:{main:{templateUrl:"partials/taskmanager/index.html",controller:"AllTaskManagersController"}}}).state("single-manager",{url:"/taskmanager/{taskmanagerid}","abstract":!0,views:{main:{templateUrl:"partials/taskmanager/taskmanager.html",controller:"SingleTaskManagerController"}}}).state("single-manager.metrics",{url:"/metrics",views:{details:{templateUrl:"partials/taskmanager/taskmanager.metrics.html"}}}).state("single-manager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/taskmanager/taskmanager.stdout.html",controller:"SingleTaskManagerStdoutController"}}}).state("single-manager.log",{url:"/log",views:{details:{templateUrl:"partials/taskmanager/taskmanager.log.html",controller:
 "SingleTaskManagerLogsController"}}}).state("jobmanager",{url:"/jobmanager",views:{main:{templateUrl:"partials/jobmanager/index.html"}}}).state("jobmanager.config",{url:"/config",views:{details:{templateUrl:"partials/jobmanager/config.html",controller:"JobManagerConfigController"}}}).state("jobmanager.stdout",{url:"/stdout",views:{details:{templateUrl:"partials/jobmanager/stdout.html",controller:"JobManagerStdoutController"}}}).state("jobmanager.log",{url:"/log",views:{details:{templateUrl:"partials/jobmanager/log.html",controller:"JobManagerLogsController"}}}).state("submit",{url:"/submit",views:{main:{templateUrl:"partials/submit.html",controller:"JobSubmitController"}}}),t.otherwise("/overview")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getLabelCla
 ss=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"<span title='{{status}}' ng-class='getBackPressureLabelClass()'><ng-transclude></ng-transclude></span>",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"<i title='{{status}}' ng-class='getLabelClass()' />",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"<td title=\"{{value || 'None'}}\">{{value || 'None'}}</td>"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",funct
 ion(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,s,a;return"undefined"==typeof e||null===e?"":(i=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),o=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===o?0===s?i+"ms":s+"s ":o+"m "+s+"s":t?n+"h "+o+"m":n+"h "+o+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/&gt;/g,">").replace(/<br\/>/g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),o>e?(e/o).toFixed(2)+" "+r[n]:1e3*o>e?(e/o).toPrecision(3)+" "+r[n]:t(e,n+1)},"undefi
 ned"==typeof e||null===e?"":1e3>e?e+" B":t(e,1)}}).filter("toLocaleString",function(){return function(e){return e.toLocaleString()}}).filter("toUpperCase",function(){return function(e){return e.toUpperCase()}}).filter("percentage",function(){return function(e){return(100*e).toFixed(0)+"%"}}),angular.module("flinkApp").service("MainService",["$http","flinkConfig","$q",function(e,t,r){return this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"config").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this}]),angular.module("flinkApp").controller("JobManagerConfigController",["$scope","JobManagerConfigService",function(e,t){return t.loadConfig().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.config=t})}]).controller("JobManagerLogsController",["$scope","JobManagerLogsService",function(e,t){return t.loadLogs().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.log=t}),e.reloadData=function(){return t.loadLog
 s().then(function(t){return e.jobmanager.log=t})}}]).controller("JobManagerStdoutController",["$scope","JobManagerStdoutService",function(e,t){return t.loadStdout().then(function(t){return null==e.jobmanager&&(e.jobmanager={}),e.jobmanager.stdout=t}),e.reloadData=function(){return t.loadStdout().then(function(t){return e.jobmanager.stdout=t})}}]),angular.module("flinkApp").service("JobManagerConfigService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadConfig=function(){var n;return n=r.defer(),e.get(t.jobServer+"jobmanager/config").success(function(e,t,r,o){return o=e,n.resolve(e)}),n.promise},this}]).service("JobManagerLogsService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadLogs=function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/log").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]).service("JobManagerStdoutService",["$http","flinkConfig","$q",function(e,t,r){var n;return n={},this.loadStdout=
 function(){var o;return o=r.defer(),e.get(t.jobServer+"jobmanager/stdout").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("RunningJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("running")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("CompletedJobsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return e.jobObserver=function(){return e.jobs=n.getJobs("finished")},n.registerObserver(e.jobObserver),e.$on("$destroy",function(){return n.unRegisterObserver(e.jobObserver)}),e.jobObserver()}]).controller("SingleJobController",["$scope","$state","$stateParams","JobsService","MetricsService","$rootScope","flinkConfig","$interval",function(e,t,r,n,o,i,s,a){var l;return e.jobid=r.jobid,e.job=null,e.plan=null,e.vertices=null,
 e.backPressureOperatorStats={},n.loadJob(r.jobid).then(function(t){return e.job=t,e.plan=t.plan,e.vertices=t.vertices,o.setupMetrics(r.jobid,t.vertices)}),l=a(function(){return n.loadJob(r.jobid).then(function(t){return e.job=t,e.$broadcast("reload")})},s["refresh-interval"]),e.$on("$destroy",function(){return e.job=null,e.plan=null,e.vertices=null,e.backPressureOperatorStats=null,a.cancel(l)}),e.cancelJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Cancelling..."),n.cancelJob(r.jobid).then(function(e){return{}})},e.stopJob=function(e){return angular.element(e.currentTarget).removeClass("btn").removeClass("btn-default").html("Stopping..."),n.stopJob(r.jobid).then(function(e){return{}})}}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.verte
 x=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){return e.nodeid?r():void 0})}]).controller("JobPlanTaskManagersController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){return e.
 nodeid?r():void 0})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){return e.nodeid?r():void 0})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){return null!==t?e.checkpointStats=t:void 0})},o(),e.$on("reload",function(e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t
 ){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){return null!==t?e.subtaskDetails[r]=t:void 0})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){return o(r.checkpointId),e.nodeid?i(r.checkpointId,e.nodeid):void 0}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){return e.now=Date.now(),e.nodeid?t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t}):void 0},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$s
 tate","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n;return e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),n=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t,e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},e.dropped=function(t,o,i,s,a){return r.orderMetrics(e.jobid,e.nodeid,i,o),e.$broadcast("metr
 ics:refresh",i),n(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),n()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),n()},e.setMetricSize=function(t,o){return r.setMetricSize(e.jobid,e.nodeid,t,o),n()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){return e.dragging?void 0:n()}),e.nodeid?n():void 0}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"<svg class='timeline secondary' width='0' height='0'></svg>",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLO
 YING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"<svg class='timeline' width='0' height='0'></svg>",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,s,a;s=r.children()[0],i=r.width(),angular.element(s).attr("width",i),a=function(e){return e.replace("&gt;",">")},o=function(r){var n,o,i;return d3.select(s).selectAll("*").remove(),i=[],angular.forEac
 h(r,function(e){return e["start-time"]>-1?"scheduled"===e.type?i.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):i.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]}):void 0}),n=d3.timeline().stack().click(function(r,n,o){return r.link?e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link}):void 0}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(s).datum(i).call(n)},t.$watch(n.vertices,function(e){return e?o(e):void 0})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"<svg class='graph' 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(e,t,r){var n,o,i,s,a,l,u,c,d,f,p,g,m,v,b,h,k,j,S,C,w,$;p=null,S=d3.behavior.zoom(),$=[],v=r.jobid,k=t.children()[0],h=t.children().children()[0],j=t.children()[1],l=d3.select(k),u=d3.select(h),c=d3.select(j),n=t.width(),angular.element(t.children()[0]).width(n),e.zoomIn=function(){var e,t,r;return S.scale()<2.99?(e=S.translate(),t=e[0]*(S.scale()+.1/S.scale()),r=e[1]*(S.scale()+.1/S.scale()),S.scale(S.scale()+.1),S.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+S.scale()+")")):void 0},e.zoomOut=function(){var e,t,r;return S.scale()>.31?(S.scale(S.scale()-.1),e=S.translate(),t=e[0]*(S.scale()-.1/S.scale()),r=e[1]*(S.scale()-.1/S.scale()),S.translate([t,r]),u.a
 ttr("transform","translate("+t+","+r+") scale("+S.scale()+")")):void 0},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="<div class='edge-label'>",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",<br>"+e.local_strategy),t+="</div>"),t},m=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},g=function(e,t){return"mirror"===t?"node-mirror":m(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var o,i;return o="<div href='#/jobs/"+v+"/vertex/"+e.id+"' class='node-label "+g(e,t)+"'>",o+="mirror"===t?"<h3 class='node-name'>Mirror of "+e.operator+"</h3>":"<h3 class='node-name'>"+e.operator+"</h3>",""===e.description?o+="":(i=e.description,i=w(i),o+="<h4 class='step-name'>"+i+"</h4>"),null!=e.step_function?o+=f(e.id,r,n):(m(t)&&(o+="<h5>"+t+" Node</h5>"),""!==e.parallelism&&(o+="<h5>Paralle
 lism: "+e.parallelism+"</h5>"),void 0!==e.operator&&e.operator_strategy&&(o+="<h5>Operation: "+w(e.operator_strategy)+"</h5>")),o+="</div>"},f=function(e,t,r){var n,o;return o="svg-"+e,n="<svg class='"+o+"' width="+t+" height="+r+"><g /></svg>"},w=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","&lt;"),e=e.replace(">","&gt;")),t="";e.length>30;)t=t+e.substring(0,30)+"<br>",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,o,i){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",o,i),labelType:"html","class":g(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",o,i),labelType:"html","class":g(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",o,i),labelType:"html","class":g(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",o,i),labelType:"html","class":g(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,
 "solutionSet",o,i),labelType:"html","class":g(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",o,i),labelType:"html","class":g(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",o,i),labelType:"html","class":g(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},b=function(e,t){var r,n,i,s,l,u,d,f,p,g,m,v,h,k;for(n=[],null!=t.nodes?k=t.nodes:(k=t.step_function,i=!0),s=0,u=k.length;u>s;s++)if(r=k[s],p=0,f=0,r.step_function&&(h=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:20,edgesep:0,ranksep:20,rankdir:"LR",marginx:10,marginy:10}),$[r.id]=h,b(h,r),m=new dagreD3.render,c.select("g").call(m,h),p=h.graph().width,f=h.graph().height,angular.element(j).empty()),a(e,t,r,i,p,f),n.push(r.id),null!=r.inputs)for(v=r.inputs,l=0,d=v.length;d>l;l++)g=v[l],o(e,t,r,n,g);return e},C=function(e,t){var r,n,o;for(n in e.nodes){if(r=e.nodes[n],r.id===t)return r;if(null!=r.step_function)
 for(o in r.step_function)if(r.step_function[o].id===t)return r.step_function[o]}},d=function(t){var r,n,o,i,s,a;p=new dagreD3.graphlib.Graph({multigraph:!0,compound:!0}).setGraph({nodesep:70,edgesep:0,ranksep:50,rankdir:"LR",marginx:40,marginy:40}),b(p,t),o=new dagreD3.render,u.call(o,p);for(r in $)i=$[r],l.select("svg.svg-"+r+" g").call(o,i);return n=.5,s=Math.floor((angular.element(k).width()-p.graph().width*n)/2),a=Math.floor((angular.element(k).height()-p.graph().height*n)/2),S.scale(n).translate([s,a]),u.attr("transform","translate("+s+", "+a+") scale("+S.scale()+")"),S.on("zoom",function(){var e;return e=d3.event,u.attr("transform","translate("+e.translate+") scale("+e.scale+")")}),S(l),u.selectAll(".node").on("click",function(t){return e.setNode({nodeid:t})})},e.$watch(r.plan,function(e){return e?d(e):void 0})}}}]),angular.module("flinkApp").service("JobsService",["$http","flinkConfig","$log","amMoment","$q","$timeout",function(e,t,r,n,o,i){var s,a,l,u,c,d;return s=null,a=nul
 l,l={},c={running:[],finished:[],cancelled:[],failed:[]},u=[],d=function(){return angular.forEach(u,function(e){return e()})},this.registerObserver=function(e){return u.push(e)},this.unRegisterObserver=function(e){var t;return t=u.indexOf(e),u.splice(t,1)},this.stateList=function(){return["SCHEDULED","DEPLOYING","RUNNING","FINISHED","FAILED","CANCELING","CANCELED"]},this.translateLabelState=function(e){switch(e.toLowerCase()){case"finished":return"success";case"failed":return"danger";case"scheduled":return"default";case"deploying":return"info";case"running":return"primary";case"canceling":return"warning";case"pending":return"info";case"total":return"black";default:return"default"}},this.setEndTimes=function(e){return angular.forEach(e,function(e,t){return e["end-time"]>-1?void 0:e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.time
 stamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"joboverview").success(function(e){return function(t,n,o,i){return angular.forEach(t,function(t,r){switch(r){case"running":return c.running=e.setEndTimes(t);case"finished":return c.finished=e.setEndTimes(t);case"cancelled":return c.cancelled=e.setEndTimes(t);case"failed":return c.failed=e.setEndTimes(t)}}),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,a,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),s=o,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,o,i,s;for(n=0,o=t.length;o>n;n++){if(i=t[n],i.id===e)return 
 i;if(i.step_function&&(s=r(e,i.step_function)),s)return s}return null},t=o.defer(),l.job.promise.then(function(n){return function(o){var i;return i=r(e,s.plan.nodes),i.vertex=n.seekVertex(e),t.resolve(i)}}(this)),t.promise},this.seekVertex=function(e){var t,r,n,o;for(n=s.vertices,t=0,r=n.length;r>t;t++)if(o=n[t],o.id===e)return o;return null},this.getVertex=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(i){var a;return a=o.seekVertex(r),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasktimes").success(function(e){return a.subtasks=e.subtasks,n.resolve(a)})}}(this)),n.promise},this.getSubtasks=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r).success(function(e){var t;return t=e.subtasks,n.resolve(t)})}}(this)),n.promise},this.getTaskManagers=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServe
 r+"jobs/"+s.jid+"/vertices/"+r+"/taskmanagers").success(function(e){var t;return t=e.taskmanagers,n.resolve(t)})}}(this)),n.promise},this.getAccumulators=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return console.log(s.jid),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/accumulators").success(function(o){var i;return i=o["user-accumulators"],e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/subtasks/accumulators").success(function(e){var t;return t=e.subtasks,n.resolve({main:i,subtasks:t})})})}}(this)),n.promise},this.getCheckpointConfig=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/config").success(function(e){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointStats=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints
 ").success(function(e,t,n,o){return angular.equals({},e)?r.resolve(null):r.resolve(e)})}}(this)),r.promise},this.getCheckpointDetails=function(r){var n;return n=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r).success(function(e){return angular.equals({},e)?n.resolve(null):n.resolve(e)})}}(this)),n.promise},this.getCheckpointSubtaskDetails=function(r,n){var i;return i=o.defer(),l.job.promise.then(function(o){return function(o){return e.get(t.jobServer+"jobs/"+s.jid+"/checkpoints/details/"+r+"/subtasks/"+n).success(function(e){return angular.equals({},e)?i.resolve(null):i.resolve(e)})}}(this)),i.promise},this.getOperatorBackPressure=function(r){var n;return n=o.defer(),e.get(t.jobServer+"jobs/"+s.jid+"/vertices/"+r+"/backpressure").success(function(e){return function(e){return n.resolve(e)}}(this)),n.promise},this.translateBackPressureLabelState=function(e){switch(e.toLowerCase()){case"in-progress":return"da
 nger";case"ok":return"success";case"low":return"warning";case"high":return"danger";default:return"default"}},this.loadExceptions=function(){var r;return r=o.defer(),l.job.promise.then(function(n){return function(n){return e.get(t.jobServer+"jobs/"+s.jid+"/exceptions").success(function(e){return s.exceptions=e,r.resolve(e)})}}(this)),r.promise},this.cancelJob=function(r){return e.get(t.jobServer+"jobs/"+r+"/yarn-cancel")},this.stopJob=function(t){return e.get("jobs/"+t+"/yarn-stop")},this}]),angular.module("flinkApp").directive("metricsGraph",function(){return{template:'<div class="panel panel-default panel-metric"> <div class="panel-heading"> <span class="metric-title">{{metric.id}}</span> <div class="buttons"> <div class="btn-group"> <button type="button" ng-class="[btnClasses, {active: metric.size != \'big\'}]" ng-click="setSize(\'small\')">Small</button> <button type="button" ng-class="[btnClasses, {active: metric.size == \'big\'}]" ng-click="setSize(\'big\')">Big</button> </div>
  <a title="Remove" class="btn btn-default btn-xs remove" ng-click="removeMetric()"><i class="fa fa-close" /></a> </div> </div> <div class="panel-body"> <svg /> </div> </div>',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&50>n;)Math.pow(10,n)<=t&&t<Math.pow(10,n+o)?r=!0:n+=o;return r&&n>6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGene
 rator(function(e){return"<p>"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"</p>"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),e.showChart(),e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;return o=[],angular.forEa
 ch(t,function(e,t){return o.push(e.id)}),o.length>0?e.getMetrics(r,n,o).then(function(t){return r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback?e.observer.callback(t):void 0;
+}):void 0})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){return r.id?t.watched[e].push(r.id):void 0}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){return null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow()?this.values[e][t].shift():void 0},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]
 :null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){return null!=e.values[r]?n.push({x:e.timestamp,y:e.values[r]}):void 0}}(this)),n)},this.setupLSFor=function(e,t){return null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t]?this.metrics[e][t]=[]:void 0},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;return null!=e.metrics[t][r]?(o=e.metrics[t][r].indexOf(n),-1===o&&(o=_.findIndex(e.metrics[t][r],{id:n})),-1!==o&&e.metrics[t][r].splice(o,1),e.saveSetup()):void 0}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;return null!=e.metrics[t][r]?(i=e.metrics[t][r].indexOf(n.id),-1===i&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),-1!==i&&(e.metrics[t][r][i]={id:n.id,size:o}),e.saveSetup()):void 0}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.
 forEach(this.metrics[e][t],function(o){return function(i,s){return i.id===r.id&&(o.metrics[e][t].splice(s,1),n>s)?n-=1:void 0}}(this)),this.metrics[e][t].splice(n,0,r),this.saveSetup()},this.getMetricsSetup=function(e){return function(t,r){return{names:_.map(e.metrics[t][r],function(e){return _.isString(e)?{id:e,size:"small"}:e})}}}(this),this.getAvailableMetrics=function(n){return function(o,i){var s;return n.setupLSFor(o,i),s=t.defer(),e.get(r.jobServer+"jobs/"+o+"/vertices/"+i+"/metrics").success(function(e){var t;return t=[],angular.forEach(e,function(e,r){var s;return s=n.metrics[o][i].indexOf(e.id),-1===s&&(s=_.findIndex(n.metrics[o][i],{id:e.id})),-1===s?t.push(e):void 0}),s.resolve(t)}),s.promise}}(this),this.getAllAvailableMetrics=function(n){return function(n,o){var i;return i=t.defer(),e.get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics").success(function(e){return i.resolve(e)}),i.promise}}(this),this.getMetrics=function(n,o,i){var s,a;return s=t.defer(),a=i.join(","),e
 .get(r.jobServer+"jobs/"+n+"/vertices/"+o+"/metrics?get="+a).success(function(e){return function(t){var r,i;return i={},angular.forEach(t,function(e,t){return i[e.id]=parseInt(e.value)}),r={timestamp:Date.now(),values:i},e.saveValue(n,o,r),s.resolve(r)}}(this)),s.promise},this.setupLS(),this}]),angular.module("flinkApp").controller("OverviewController",["$scope","OverviewService","JobsService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.jobObserver=function(){return e.runningJobs=r.getJobs("running"),e.finishedJobs=r.getJobs("finished")},r.registerObserver(e.jobObserver),e.$on("$destroy",function(){return r.unRegisterObserver(e.jobObserver)}),e.jobObserver(),t.loadOverview().then(function(t){return e.overview=t}),i=n(function(){return t.loadOverview().then(function(t){return e.overview=t})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]),angular.module("flinkApp").service("OverviewService",["$http","flinkConfig","$q",function(e,t,r){var n;
 return n={},this.loadOverview=function(){var o;return o=r.defer(),e.get(t.jobServer+"overview").success(function(e,t,r,i){return n=e,o.resolve(e)}),o.promise},this}]),angular.module("flinkApp").controller("JobSubmitController",["$scope","JobSubmitService","$interval","flinkConfig","$state","$location",function(e,t,r,n,o,i){var s;return e.yarn=-1!==i.absUrl().indexOf("/proxy/application_"),e.loadList=function(){return t.loadJarList().then(function(t){return e.address=t.address,e.noaccess=t.error,e.jars=t.files})},e.defaultState=function(){return e.plan=null,e.error=null,e.state={selected:null,parallelism:"",savepointPath:"",allowNonRestoredState:!1,"entry-class":"","program-args":"","plan-button":"Show Plan","submit-button":"Submit","action-time":0}},e.defaultState(),e.uploader={},e.loadList(),s=r(function(){return e.loadList()},n["refresh-interval"]),e.$on("$destroy",function(){return r.cancel(s)}),e.selectJar=function(t){return e.state.selected===t?e.defaultState():(e.defaultState(
 ),e.state.selected=t)},e.deleteJar=function(r,n){return e.state.selected===n&&e.defaultState(),angular.element(r.currentTarget).removeClass("fa-remove").addClass("fa-spin fa-spinner"),t.deleteJar(n).then(function(e){return angular.element(r.currentTarget).removeClass("fa-spin fa-spinner").addClass("fa-remove"),null!=e.error?alert(e.error):void 0})},e.loadEntryClass=function(t){return e.state["entry-class"]=t},e.getPlan=function(){var r;return"Show Plan"===e.state["plan-button"]?(r=(new Date).getTime(),e.state["action-time"]=r,e.state["submit-button"]="Submit",e.state["plan-button"]="Getting Plan",e.error=null,e.plan=null,t.getPlan(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"]}).then(function(t){return r===e.state["action-time"]?(e.state["plan-button"]="Show Plan",e.error=t.error,e.plan=t.plan):void 0})):void 0},e.runJob=function(){var r;return"Submit"===e.state["submit-button"]?(r=(new Date).getTime(),e.
 state["action-time"]=r,e.state["submit-button"]="Submitting",e.state["plan-button"]="Show Plan",e.error=null,t.runJob(e.state.selected,{"entry-class":e.state["entry-class"],parallelism:e.state.parallelism,"program-args":e.state["program-args"],savepointPath:e.state.savepointPath,allowNonRestoredState:e.state.allowNonRestoredState}).then(function(t){return r===e.state["action-time"]&&(e.state["submit-button"]="Submit",e.error=t.error,null!=t.jobid)?o.go("single-job.plan.subtasks",{jobid:t.jobid}):void 0})):void 0},e.nodeid=null,e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.$broadcast("reload")):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null)},e.clearFiles=function(){return e.uploader={}},e.uploadFiles=function(t){return e.uploader={},1===t.length?(e.uploader.file=t[0],e.uploader.upload=!0):e.uploader.error="Did ya forget to select a file?"},e.startUpload=function(){var t,r;return null!=
 e.uploader.file?(t=new FormData,t.append("jarfile",e.uploader.file),e.uploader.upload=!1,e.uploader.success="Initializing upload...",r=new XMLHttpRequest,r.upload.onprogress=function(t){return e.uploader.success=null,e.uploader.progress=parseInt(100*t.loaded/t.total)},r.upload.onerror=function(t){return e.uploader.progress=null,e.uploader.error="An error occurred while uploading your file"},r.upload.onload=function(t){return e.uploader.progress=null,e.uploader.success="Saving..."},r.onreadystatechange=function(){var t;return 4===r.readyState?(t=JSON.parse(r.responseText),null!=t.error?(e.uploader.error=t.error,e.uploader.success=null):e.uploader.success="Uploaded!"):void 0},r.open("POST",n.jobServer+"jars/upload"),r.send(t)):console.log("Unexpected Error. This should not happen")}}]).filter("getJarSelectClass",function(){return function(e,t){return e===t?"fa-check-square":"fa-square-o"}}),angular.module("flinkApp").service("JobSubmitService",["$http","flinkConfig","$q",function(e,t,
 r){return this.loadJarList=function(){var n;return n=r.defer(),e.get(t.jobServer+"jars/").success(function(e,t,r,o){return n.resolve(e)}),n.promise},this.deleteJar=function(n){var o;return o=r.defer(),e["delete"](t.jobServer+"jars/"+encodeURIComponent(n)).success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.getPlan=function(n,o){var i;return i=r.defer(),e.get(t.jobServer+"jars/"+encodeURIComponent(n)+"/plan",{params:o}).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this.runJob=function(n,o){var i;return i=r.defer(),e.post(t.jobServer+"jars/"+encodeURIComponent(n)+"/run",{},{params:o}).success(function(e,t,r,n){return i.resolve(e)}),i.promise},this}]),angular.module("flinkApp").controller("AllTaskManagersController",["$scope","TaskManagersService","$interval","flinkConfig",function(e,t,r,n){var o;return t.loadManagers().then(function(t){return e.managers=t}),o=r(function(){return t.loadManagers().then(function(t){return e.managers=t})},n["refresh-interval"]),e
 .$on("$destroy",function(){return r.cancel(o)})}]).controller("SingleTaskManagerController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){var i;return e.metrics={},r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]}),i=n(function(){return r.loadMetrics(t.taskmanagerid).then(function(t){return e.metrics=t[0]})},o["refresh-interval"]),e.$on("$destroy",function(){return n.cancel(i)})}]).controller("SingleTaskManagerLogsController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.log={},e.taskmanagerid=t.taskmanagerid,r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t}),e.reloadData=function(){return r.loadLogs(t.taskmanagerid).then(function(t){return e.log=t})}}]).controller("SingleTaskManagerStdoutController",["$scope","$stateParams","SingleTaskManagerService","$interval","flinkConfig",function(e,t,r,n,o){return e.stdout={},e.taskmanagerid=t.taskman
 agerid,r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t}),e.reloadData=function(){return r.loadStdout(t.taskmanagerid).then(function(t){return e.stdout=t})}}]),angular.module("flinkApp").service("TaskManagersService",["$http","flinkConfig","$q",function(e,t,r){return this.loadManagers=function(){var n;return n=r.defer(),e.get(t.jobServer+"taskmanagers").success(function(e,t,r,o){return n.resolve(e.taskmanagers)}),n.promise},this}]).service("SingleTaskManagerService",["$http","flinkConfig","$q",function(e,t,r){return this.loadMetrics=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n).success(function(e,t,r,n){return o.resolve(e.taskmanagers)}),o.promise},this.loadLogs=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/log").success(function(e,t,r,n){return o.resolve(e)}),o.promise},this.loadStdout=function(n){var o;return o=r.defer(),e.get(t.jobServer+"taskmanagers/"+n+"/stdout").success(function(e,t,r,n){return o.resolve
 (e)}),o.promise},this}]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/080617d2/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.metrics.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.metrics.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.metrics.html
index a780f31..26a1e65 100644
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.metrics.html
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.plan.node-list.metrics.html
@@ -21,16 +21,16 @@ limitations under the License.
 <nav ng-if="nodeid" class="navbar navbar-default navbar-secondary-additional navbar-secondary-additional-2">
   <div class="navbar-info">{{ vertex.name }}</div>
   <div class="navbar-info">{{ nodeid }}</div>
-  <div ng-if="availableMetrics.length" class="dropup add-metrics">
+  <div ng-if="availableMetrics.length" class="dropdown add-metrics">
     <button type="button" data-toggle="dropdown" class="btn btn-default navbar-btn dropdown-toggle">
       Add metric
       &nbsp;<span class="caret"></span>
     </button>
     <ul class="dropdown-menu dropdown-menu-right metric-menu">
-      <li ng-repeat="metric in availableMetrics track by $index"><a ng-click="addMetric(metric)">{{ metric.id }}</a></li>
+      <li ng-repeat="metric in availableMetrics track by $index"><a ng-click="addMetric(metric)">{{ metric.id | limit }}</a></li>
     </ul>
   </div>
-  <div ng-if="!availableMetrics.length" class="dropup add-metrics">
+  <div ng-if="!availableMetrics.length" class="dropdown add-metrics">
     <button type="button" data-toggle="dropdown" disabled="disabled" class="btn btn-default navbar-btn dropdown-toggle"><i>No metrics available</i></button>
   </div>
 </nav>


[07/17] flink git commit: [FLINK-5424] Improve Restart Strategy Logging

Posted by ch...@apache.org.
[FLINK-5424] Improve Restart Strategy Logging

- Added toString for FailureRateRestartStrategy (important for
JobManager's "Using restart strategy $restartStrategy" log message)
- Added explanation in log when the restart strategy is responsible
for preventing job restart


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

Branch: refs/heads/release-1.2
Commit: fc4b45e4781a4ccd595ad31aefeba448fdea9700
Parents: f90edc4
Author: Shannon Carey <re...@gmail.com>
Authored: Fri Jan 6 13:25:52 2017 -0600
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:01 2017 +0100

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionGraph.java      | 16 ++++++++++++++--
 .../restart/FailureRateRestartStrategy.java         | 10 ++++++++++
 2 files changed, 24 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fc4b45e4/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 058872a..f31eada 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.flink.api.common.Archiveable;
 import org.apache.flink.api.common.ArchivedExecutionConfig;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -1049,7 +1050,9 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 					LOG.info("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID());
 				}
 
-				boolean isRestartable = !(failureCause instanceof SuppressRestartsException) && restartStrategy.canRestart();
+				final boolean isFailureCauseAllowingRestart = !(failureCause instanceof SuppressRestartsException);
+				final boolean isRestartStrategyAllowingRestart = restartStrategy.canRestart();
+				boolean isRestartable = isFailureCauseAllowingRestart && isRestartStrategyAllowingRestart;
 
 				if (isRestartable && transitionState(currentState, JobStatus.RESTARTING)) {
 					LOG.info("Restarting the job {} ({}).", getJobName(), getJobID());
@@ -1057,7 +1060,16 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 					return true;
 				} else if (!isRestartable && transitionState(currentState, JobStatus.FAILED, failureCause)) {
-					LOG.info("Could not restart the job {} ({}).", getJobName(), getJobID(), failureCause);
+					final List<String> reasonsForNoRestart = new ArrayList<>(2);
+					if (!isFailureCauseAllowingRestart) {
+						reasonsForNoRestart.add("a type of SuppressRestartsException was thrown");
+					}
+					if (!isRestartStrategyAllowingRestart) {
+						reasonsForNoRestart.add("the restart strategy prevented it");
+					}
+
+					LOG.info("Could not restart the job {} ({}) because {}.", getJobName(), getJobID(),
+						StringUtils.join(reasonsForNoRestart, " and "), failureCause);
 					postRunCleanup();
 
 					return true;

http://git-wip-us.apache.org/repos/asf/flink/blob/fc4b45e4/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
index 10546a2..d95e1c3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.util.Preconditions;
+
 import scala.concurrent.duration.Duration;
 
 import java.util.ArrayDeque;
@@ -77,6 +78,15 @@ public class FailureRateRestartStrategy implements RestartStrategy {
 		return restartTimestampsDeque.size() == maxFailuresPerInterval;
 	}
 
+	@Override
+	public String toString() {
+		return "FailureRateRestartStrategy(" +
+			"failuresInterval=" + failuresInterval +
+			"delayInterval=" + delayInterval +
+			"maxFailuresPerInterval=" + maxFailuresPerInterval +
+			")";
+	}
+
 	public static FailureRateRestartStrategyFactory createFactory(Configuration configuration) throws Exception {
 		int maxFailuresPerInterval = configuration.getInteger(ConfigConstants.RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL, 1);
 		String failuresIntervalString = configuration.getString(


[13/17] flink git commit: [FLINK-5113] Port functions in tests to new CheckpointedFunction IF.

Posted by ch...@apache.org.
[FLINK-5113] Port functions in tests to new CheckpointedFunction IF.


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

Branch: refs/heads/release-1.2
Commit: 8b069fde3adccdcd5143de90d3d4834f33b5acff
Parents: 2bbc92c
Author: kl0u <kk...@gmail.com>
Authored: Tue Nov 22 19:23:33 2016 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 23:59:09 2017 +0100

----------------------------------------------------------------------
 .../CassandraTupleWriteAheadSinkExample.java    |  22 ++-
 .../fs/RollingSinkFaultToleranceITCase.java     |  18 ++-
 .../BucketingSinkFaultToleranceITCase.java      |  18 ++-
 .../connectors/kafka/KafkaConsumerTestBase.java |  22 ++-
 .../kafka/testutils/FailingIdentityMapper.java  |  18 ++-
 .../testutils/ValidatingExactlyOnceSink.java    |  26 ++--
 .../org/apache/flink/util/CollectionUtil.java   |   2 +-
 .../streaming/api/checkpoint/Checkpointed.java  |   3 +-
 .../runtime/tasks/SourceStreamTaskTest.java     |  12 +-
 ...tractEventTimeWindowCheckpointingITCase.java |  43 ++++--
 .../CoStreamCheckpointingITCase.java            |  74 ++++++----
 .../EventTimeAllWindowCheckpointingITCase.java  |  38 +++--
 .../PartitionedStateCheckpointingITCase.java    |  17 ++-
 .../test/checkpointing/RescalingITCase.java     |  27 ++--
 .../test/checkpointing/SavepointITCase.java     |  14 +-
 .../checkpointing/StateCheckpointedITCase.java  |  94 +++++++-----
 .../StreamCheckpointNotifierITCase.java         |  31 ++--
 .../StreamCheckpointingITCase.java              |  79 ++++++----
 .../UdfStreamOperatorCheckpointingITCase.java   |  14 +-
 .../WindowCheckpointingITCase.java              |  33 +++--
 .../jar/CheckpointedStreamingProgram.java       |  28 ++--
 .../jar/CheckpointingCustomKvStateProgram.java  |  21 +--
 .../jar/LegacyCheckpointedStreamingProgram.java | 148 +++++++++++++++++++
 .../flink/test/recovery/ChaosMonkeyITCase.java  |  38 +++--
 .../JobManagerHACheckpointRecoveryITCase.java   |  41 +++--
 ...erProcessFailureStreamingRecoveryITCase.java |  30 ++--
 26 files changed, 608 insertions(+), 303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
index 811c410..23de949 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
@@ -21,12 +21,14 @@ import com.datastax.driver.core.Cluster;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.connectors.cassandra.CassandraSink;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 
+import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -50,6 +52,9 @@ public class CassandraTupleWriteAheadSinkExample {
 			.setQuery("INSERT INTO example.values (id, counter) values (?, ?);")
 			.enableWriteAheadLog()
 			.setClusterBuilder(new ClusterBuilder() {
+
+				private static final long serialVersionUID = 2793938419775311824L;
+
 				@Override
 				public Cluster buildCluster(Cluster.Builder builder) {
 					return builder.addContactPoint("127.0.0.1").build();
@@ -62,7 +67,9 @@ public class CassandraTupleWriteAheadSinkExample {
 		env.execute();
 	}
 
-	public static class MySource implements SourceFunction<Tuple2<String, Integer>>, Checkpointed<Integer> {
+	public static class MySource implements SourceFunction<Tuple2<String, Integer>>, ListCheckpointed<Integer> {
+		private static final long serialVersionUID = 4022367939215095610L;
+
 		private int counter = 0;
 		private boolean stop = false;
 
@@ -84,13 +91,16 @@ public class CassandraTupleWriteAheadSinkExample {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return counter;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.counter);
 		}
 
 		@Override
-		public void restoreState(Integer state) throws Exception {
-			this.counter = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.counter = state.get(0);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
index 36c0d03..2d8492f 100644
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.fs;
 
 import com.google.common.collect.Sets;
 import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
@@ -42,7 +42,9 @@ import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Matcher;
@@ -236,7 +238,7 @@ public class RollingSinkFaultToleranceITCase extends StreamFaultToleranceTestBas
 	}
 
 	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
-			implements CheckpointedAsynchronously<Integer> {
+			implements ListCheckpointed<Integer> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -246,7 +248,6 @@ public class RollingSinkFaultToleranceITCase extends StreamFaultToleranceTestBas
 
 		private volatile boolean isRunning = true;
 
-
 		StringGeneratingSourceFunction(long numElements) {
 			this.numElements = numElements;
 		}
@@ -288,13 +289,16 @@ public class RollingSinkFaultToleranceITCase extends StreamFaultToleranceTestBas
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
index 54703a3..85f23b6 100644
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.fs.bucketing;
 
 import com.google.common.collect.Sets;
 import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
@@ -42,7 +42,9 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.BufferedReader;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Matcher;
@@ -233,7 +235,7 @@ public class BucketingSinkFaultToleranceITCase extends StreamFaultToleranceTestB
 	}
 
 	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
-			implements CheckpointedAsynchronously<Integer> {
+			implements ListCheckpointed<Integer> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -243,7 +245,6 @@ public class BucketingSinkFaultToleranceITCase extends StreamFaultToleranceTestB
 
 		private volatile boolean isRunning = true;
 
-
 		StringGeneratingSourceFunction(long numElements) {
 			this.numElements = numElements;
 		}
@@ -285,13 +286,16 @@ public class BucketingSinkFaultToleranceITCase extends StreamFaultToleranceTestB
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index aa7ea49..d7fab88 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -51,21 +51,16 @@ import org.apache.flink.runtime.client.JobCancellationException;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
 import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
@@ -73,7 +68,6 @@ import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidating
 import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
@@ -1925,7 +1919,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 
 	public static class BrokerKillingMapper<T> extends RichMapFunction<T,T>
-			implements Checkpointed<Integer>, CheckpointListener {
+			implements ListCheckpointed<Integer>, CheckpointListener {
 
 		private static final long serialVersionUID = 6334389850158707313L;
 
@@ -1939,7 +1933,6 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		private boolean failer;
 		private boolean hasBeenCheckpointed;
 
-
 		public BrokerKillingMapper(int shutdownBrokerId, int failCount) {
 			this.shutdownBrokerId = shutdownBrokerId;
 			this.failCount = failCount;
@@ -1994,13 +1987,16 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsTotal;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.numElementsTotal);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			this.numElementsTotal = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.numElementsTotal = state.get(0);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
index 2bd400c..ec64b00 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
@@ -21,13 +21,16 @@ package org.apache.flink.streaming.connectors.kafka.testutils;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collections;
+import java.util.List;
+
 
 public class FailingIdentityMapper<T> extends RichMapFunction<T,T> implements
-		Checkpointed<Integer>, CheckpointListener, Runnable {
+	ListCheckpointed<Integer>, CheckpointListener, Runnable {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(FailingIdentityMapper.class);
 	
@@ -89,13 +92,16 @@ public class FailingIdentityMapper<T> extends RichMapFunction<T,T> implements
 	}
 
 	@Override
-	public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-		return numElementsTotal;
+	public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+		return Collections.singletonList(numElementsTotal);
 	}
 
 	@Override
-	public void restoreState(Integer state) {
-		numElementsTotal = state;
+	public void restoreState(List<Integer> state) throws Exception {
+		if (state.isEmpty() || state.size() > 1) {
+			throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+		}
+		this.numElementsTotal = state.get(0);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
index 7813561..46e70fd 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
@@ -19,15 +19,17 @@
 package org.apache.flink.streaming.connectors.kafka.testutils;
 
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.test.util.SuccessException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
 
-public class ValidatingExactlyOnceSink extends RichSinkFunction<Integer> implements Checkpointed<Tuple2<Integer, BitSet>> {
+public class ValidatingExactlyOnceSink extends RichSinkFunction<Integer> implements ListCheckpointed<Tuple2<Integer, BitSet>> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class);
 
@@ -39,7 +41,6 @@ public class ValidatingExactlyOnceSink extends RichSinkFunction<Integer> impleme
 
 	private int numElements; // this is checkpointed
 
-	
 	public ValidatingExactlyOnceSink(int numElementsTotal) {
 		this.numElementsTotal = numElementsTotal;
 	}
@@ -68,15 +69,20 @@ public class ValidatingExactlyOnceSink extends RichSinkFunction<Integer> impleme
 	}
 
 	@Override
-	public Tuple2<Integer, BitSet> snapshotState(long checkpointId, long checkpointTimestamp) {
-		LOG.info("Snapshot of counter "+numElements+" at checkpoint "+checkpointId);
-		return new Tuple2<>(numElements, duplicateChecker);
+	public List<Tuple2<Integer, BitSet>> snapshotState(long checkpointId, long timestamp) throws Exception {
+		LOG.info("Snapshot of counter " + numElements + " at checkpoint " + checkpointId);
+		return Collections.singletonList(new Tuple2<>(numElements, duplicateChecker));
 	}
 
 	@Override
-	public void restoreState(Tuple2<Integer, BitSet> state) {
-		LOG.info("restoring num elements to {}", state.f0);
-		this.numElements = state.f0;
-		this.duplicateChecker = state.f1;
+	public void restoreState(List<Tuple2<Integer, BitSet>> state) throws Exception {
+		if (state.isEmpty() || state.size() > 1) {
+			throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+		}
+
+		Tuple2<Integer, BitSet> s = state.get(0);
+		LOG.info("restoring num elements to {}", s.f0);
+		this.numElements = s.f0;
+		this.duplicateChecker = s.f1;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
index 15d00ae..cd5c2e5 100644
--- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
@@ -34,4 +34,4 @@ public final class CollectionUtil {
 	public static boolean isNullOrEmpty(Map<?, ?> map) {
 		return map == null || map.isEmpty();
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
index 7af5cea..fb67ea7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
@@ -31,8 +31,7 @@ import java.io.Serializable;
  * state is written, the function is not called, so the function needs not return a
  * copy of its state, but may return a reference to its state. Functions that can
  * continue to work and mutate the state, even while the state snapshot is being accessed,
- * can implement the {@link org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously}
- * interface.</p>
+ * can implement the {@link CheckpointedAsynchronously} interface.</p>
  * 
  * @param <T> The type of the operator state.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
index b592fe8..dd1fe58 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -34,6 +34,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.Serializable;
+import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.Callable;
@@ -135,7 +136,7 @@ public class SourceStreamTaskTest {
 		}
 	}
 
-	private static class MockSource implements SourceFunction<Tuple2<Long, Integer>>, Checkpointed<Serializable> {
+	private static class MockSource implements SourceFunction<Tuple2<Long, Integer>>, ListCheckpointed<Serializable> {
 		private static final long serialVersionUID = 1;
 
 		private int maxElements;
@@ -181,7 +182,7 @@ public class SourceStreamTaskTest {
 		}
 
 		@Override
-		public Serializable snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<Serializable> snapshotState(long checkpointId, long timestamp) throws Exception {
 			if (!semaphore.tryAcquire()) {
 				Assert.fail("Concurrent invocation of snapshotState.");
 			}
@@ -199,11 +200,12 @@ public class SourceStreamTaskTest {
 				Assert.fail("Count is different at start end end of snapshot.");
 			}
 			semaphore.release();
-			return sum;
+			return Collections.<Serializable>singletonList(sum);
 		}
 
 		@Override
-		public void restoreState(Serializable state) {}
+		public void restoreState(List<Serializable> state) throws Exception {
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
index 583e42f..1911f44 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
@@ -34,7 +34,7 @@ import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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.RichSourceFunction;
@@ -53,7 +53,9 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.flink.test.util.TestUtils.tryExecute;
@@ -495,7 +497,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements Checkpointed<Integer>, CheckpointListener
+			implements ListCheckpointed<Integer>, CheckpointListener
 	{
 		private static volatile boolean failedBefore = false;
 
@@ -567,13 +569,16 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsEmitted;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.numElementsEmitted);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			numElementsEmitted = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.numElementsEmitted = state.get(0);
 		}
 
 		public static void reset() {
@@ -582,7 +587,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 	}
 
 	private static class ValidatingSink extends RichSinkFunction<Tuple4<Long, Long, Long, IntType>>
-			implements Checkpointed<HashMap<Long, Integer>> {
+			implements ListCheckpointed<HashMap<Long, Integer>> {
 
 		private final HashMap<Long, Integer> windowCounts = new HashMap<>();
 
@@ -676,19 +681,22 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		}
 
 		@Override
-		public HashMap<Long, Integer> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return this.windowCounts;
+		public List<HashMap<Long, Integer>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.windowCounts);
 		}
 
 		@Override
-		public void restoreState(HashMap<Long, Integer> state) {
-			this.windowCounts.putAll(state);
+		public void restoreState(List<HashMap<Long, Integer>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			windowCounts.putAll(state.get(0));
 		}
 	}
 
 	// Sink for validating the stateful window counts
 	private static class CountValidatingSink extends RichSinkFunction<Tuple4<Long, Long, Long, IntType>>
-			implements Checkpointed<HashMap<Long, Integer>> {
+			implements ListCheckpointed<HashMap<Long, Integer>> {
 
 		private final HashMap<Long, Integer> windowCounts = new HashMap<>();
 
@@ -757,13 +765,16 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		}
 
 		@Override
-		public HashMap<Long, Integer> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return this.windowCounts;
+		public List<HashMap<Long, Integer>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.windowCounts);
 		}
 
 		@Override
-		public void restoreState(HashMap<Long, Integer> state) {
-			this.windowCounts.putAll(state);
+		public void restoreState(List<HashMap<Long, Integer>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.windowCounts.putAll(state.get(0));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
index 51a00b9..06d3ab0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
@@ -38,6 +38,8 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
 import java.util.Random;
 
 import static org.junit.Assert.assertEquals;
@@ -142,11 +144,11 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 	/**
 	 * A generating source that is slow before the first two checkpoints went through
 	 * and will indefinitely stall at a certain point to allow the checkpoint to complete.
-	 * 
+	 *
 	 * After the checkpoints are through, it continues with full speed.
 	 */
 	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
-			implements Checkpointed<Integer>, CheckpointListener {
+			implements ListCheckpointed<Integer>, CheckpointListener {
 
 		private static volatile int numCompletedCheckpoints = 0;
 
@@ -210,13 +212,16 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 
 		@Override
@@ -239,12 +244,11 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 	}
 
 	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 
 		static final long[] counts = new long[PARALLELISM];
 		
 		private long count;
-		
 
 		@Override
 		public PrefixCount map(PrefixCount value) throws Exception {
@@ -258,13 +262,16 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 
@@ -303,7 +310,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 	}
 
-	private static class StringRichFilterFunction extends RichFilterFunction<String> implements Checkpointed<Long> {
+	private static class StringRichFilterFunction extends RichFilterFunction<String> implements ListCheckpointed<Long> {
 
 		static final long[] counts = new long[PARALLELISM];
 
@@ -321,18 +328,21 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 
-	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> implements Checkpointed<Long> {
-
+	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> implements ListCheckpointed<Long> {
+		
 		static final long[] counts = new long[PARALLELISM];
 
 		private long count;
@@ -344,13 +354,16 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 
 		@Override
@@ -359,7 +372,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 	}
 
-	private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction<String, String, String> implements Checkpointed<Long> {
+	private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction<String, String, String> implements ListCheckpointed<Long> {
 
 		static final long[] counts = new long[PARALLELISM];
 
@@ -378,13 +391,16 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
index b493e42..09c1437 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
@@ -28,7 +28,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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.RichSourceFunction;
@@ -43,14 +43,16 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.flink.test.util.TestUtils.tryExecute;
 import static org.junit.Assert.*;
 
 /**
- * This verfies that checkpointing works correctly with event time windows.
+ * This verifies that checkpointing works correctly with event time windows.
  *
  * <p>
  * This is a version of {@link AbstractEventTimeWindowCheckpointingITCase} for All-Windows.
@@ -432,7 +434,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements Checkpointed<Integer>, CheckpointListener
+			implements ListCheckpointed<Integer>, CheckpointListener
 	{
 		private static volatile boolean failedBefore = false;
 
@@ -502,23 +504,26 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			numSuccessfulCheckpoints++;
 		}
 
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsEmitted;
+		public static void reset() {
+			failedBefore = false;
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			numElementsEmitted = state;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.numElementsEmitted);
 		}
 
-		public static void reset() {
-			failedBefore = false;
+		@Override
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.numElementsEmitted = state.get(0);
 		}
 	}
 
 	private static class ValidatingSink extends RichSinkFunction<Tuple4<Long, Long, Long, IntType>>
-			implements Checkpointed<HashMap<Long, Integer>> {
+			implements ListCheckpointed<HashMap<Long, Integer>> {
 
 		private final HashMap<Long, Integer> windowCounts = new HashMap<>();
 
@@ -612,13 +617,16 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		}
 
 		@Override
-		public HashMap<Long, Integer> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return this.windowCounts;
+		public List<HashMap<Long, Integer>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.windowCounts);
 		}
 
 		@Override
-		public void restoreState(HashMap<Long, Integer> state) {
-			this.windowCounts.putAll(state);
+		public void restoreState(List<HashMap<Long, Integer>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.windowCounts.putAll(state.get(0));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
index 0728b41..4761d70 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
@@ -33,7 +35,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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;
@@ -86,7 +88,7 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
 	// --------------------------------------------------------------------------------------------
 
 	private static class IntGeneratingSourceFunction extends RichParallelSourceFunction<Integer> 
-		implements Checkpointed<Integer> {
+		implements ListCheckpointed<Integer> {
 
 		private final long numElements;
 
@@ -133,13 +135,16 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
index bd1678e..8045d82 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
@@ -41,7 +41,6 @@ import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -66,6 +65,7 @@ import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -763,7 +763,7 @@ public class RescalingITCase extends TestLogger {
 		}
 	}
 
-	private static class SubtaskIndexNonPartitionedStateSource extends SubtaskIndexSource implements Checkpointed<Integer> {
+	private static class SubtaskIndexNonPartitionedStateSource extends SubtaskIndexSource implements ListCheckpointed<Integer> {
 
 		private static final long serialVersionUID = 8388073059042040203L;
 
@@ -772,13 +772,16 @@ public class RescalingITCase extends TestLogger {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return counter;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.counter);
 		}
 
 		@Override
-		public void restoreState(Integer state) throws Exception {
-			counter = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.counter = state.get(0);
 		}
 	}
 
@@ -879,18 +882,20 @@ public class RescalingITCase extends TestLogger {
 		}
 	}
 
-	private static class NonPartitionedStateSource extends StateSourceBase implements Checkpointed<Integer> {
+	private static class NonPartitionedStateSource extends StateSourceBase implements ListCheckpointed<Integer> {
 
 		private static final long serialVersionUID = -8108185918123186841L;
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return counter;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.counter);
 		}
 
 		@Override
-		public void restoreState(Integer state) throws Exception {
-			counter = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (!state.isEmpty()) {
+				this.counter = state.get(0);
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index 9f957e5..77777d1 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -63,7 +63,6 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ResponseS
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.ResponseSubmitTaskListener;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.IterativeStream;
@@ -527,7 +526,7 @@ public class SavepointITCase extends TestLogger {
 
 	private static class StatefulCounter
 		extends RichMapFunction<Integer, Integer>
-		implements Checkpointed<byte[]>, CheckpointListener {
+		implements ListCheckpointed<byte[]>, CheckpointListener {
 
 		private static final Object checkpointLock = new Object();
 		private static int numCompleteCalls;
@@ -556,13 +555,16 @@ public class SavepointITCase extends TestLogger {
 		}
 
 		@Override
-		public byte[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return data;
+		public List<byte[]> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(data);
 		}
 
 		@Override
-		public void restoreState(byte[] data) throws Exception {
-			this.data = data;
+		public void restoreState(List<byte[]> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.data = state.get(0);
 
 			synchronized (checkpointLock) {
 				if (++numRestoreCalls == getRuntimeContext().getNumberOfParallelSubtasks()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/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
index 9d37b59..32d9e23 100644
--- 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
@@ -24,8 +24,7 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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;
@@ -34,9 +33,10 @@ import org.apache.flink.util.Collector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -47,7 +47,7 @@ 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 ValueState} or the {@link Checkpointed}
+ * state defined with either the {@link ValueState} or the {@link ListCheckpointed}
  * interface reflects the "exactly once" semantics.
  * 
  * The test throttles the input until at least two checkpoints are completed, to make sure that
@@ -139,15 +139,14 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 	// --------------------------------------------------------------------------------------------
 	
 	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String> 
-			implements CheckpointedAsynchronously<Integer>
+			implements ListCheckpointed<Integer>
 	{
 		private final long numElements;
 
 		private int index;
 
 		private volatile boolean isRunning = true;
-		
-		
+
 		StringGeneratingSourceFunction(long numElements) {
 			this.numElements = numElements;
 		}
@@ -197,23 +196,26 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 	}
 
 	private static class StringRichFilterFunction extends RichFilterFunction<String> 
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 
 		static final long[] counts = new long[PARALLELISM];
 		
 		private long count;
-		
+
 		@Override
 		public boolean filter(String value) throws Exception {
 			count++;
@@ -226,23 +228,26 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 
 	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> 
-			implements CheckpointedAsynchronously<Long> {
+			implements ListCheckpointed<Long> {
 		
 		static final long[] counts = new long[PARALLELISM];
 
 		private long count;
-		
+
 		@Override
 		public PrefixCount map(String value) {
 			count++;
@@ -255,18 +260,21 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 	
 	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
-		implements Checkpointed<Long> {
+		implements ListCheckpointed<Long> {
 
 		static final long[] counts = new long[PARALLELISM];
 		
@@ -284,18 +292,21 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 	
 	private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount> 
-		implements Checkpointed<HashMap<String, PrefixCount>>, CheckpointListener {
+		implements ListCheckpointed<HashMap<String, PrefixCount>>, CheckpointListener {
 
 		static boolean wasCheckpointedBeforeFailure = false;
 		
@@ -307,7 +318,6 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		private long count;
 		
 		private boolean wasCheckpointed;
-		
 
 		OnceFailingAggregator(long failurePos) {
 			this.failurePos = failurePos;
@@ -339,13 +349,16 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public HashMap<String, PrefixCount> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return aggregationMap;
+		public List<HashMap<String, PrefixCount>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.aggregationMap);
 		}
 
 		@Override
-		public void restoreState(HashMap<String, PrefixCount> state) {
-			aggregationMap.putAll(state);
+		public void restoreState(List<HashMap<String, PrefixCount>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.aggregationMap.putAll(state.get(0));
 		}
 
 		@Override
@@ -355,7 +368,7 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 	}
 
 	private static class ValidatingSink extends RichSinkFunction<PrefixCount> 
-			implements Checkpointed<HashMap<Character, Long>> {
+			implements ListCheckpointed<HashMap<Character, Long>> {
 
 		@SuppressWarnings("unchecked")
 		private static Map<Character, Long>[] maps = (Map<Character, Long>[]) new Map<?, ?>[PARALLELISM];
@@ -379,13 +392,16 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public HashMap<Character, Long> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return counts;
+		public List<HashMap<Character, Long>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.counts);
 		}
 
 		@Override
-		public void restoreState(HashMap<Character, Long> state) {
-			counts.putAll(state);
+		public void restoreState(List<HashMap<Character, Long>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.counts.putAll(state.get(0));
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
index 6bf511f..be3fac5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
@@ -26,7 +26,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
@@ -47,6 +47,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
@@ -205,7 +206,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 	 * interface it stores all the checkpoint ids it has seen in a static list.
 	 */
 	private static class GeneratingSourceFunction extends RichSourceFunction<Long>
-			implements ParallelSourceFunction<Long>, CheckpointListener, Checkpointed<Integer> {
+			implements ParallelSourceFunction<Long>, CheckpointListener, ListCheckpointed<Integer> {
 		
 		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
 		
@@ -263,13 +264,16 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 
 		@Override
@@ -390,7 +394,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 	 * Reducer that causes one failure between seeing 40% to 70% of the records.
 	 */
 	private static class OnceFailingReducer extends RichReduceFunction<Tuple1<Long>> 
-		implements Checkpointed<Long>, CheckpointListener
+		implements ListCheckpointed<Long>, CheckpointListener
 	{
 		static volatile boolean hasFailed = false;
 		static volatile long failureCheckpointID;
@@ -402,7 +406,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 		private volatile long count;
 
 		private volatile boolean notificationAlready;
-		
+
 		OnceFailingReducer(long numElements) {
 			this.failurePos = (long) (0.5 * numElements / PARALLELISM);
 		}
@@ -419,19 +423,22 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
 			if (!hasFailed && count >= failurePos && getRuntimeContext().getIndexOfThisSubtask() == 0) {
 				LOG.info(">>>>>>>>>>>>>>>>> Throwing Exception <<<<<<<<<<<<<<<<<<<<<");
 				hasFailed = true;
 				failureCheckpointID = checkpointId;
 				throw new Exception("Test Failure");
 			}
-			return count;
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
index b97e1f2..aae04c9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
@@ -31,6 +31,8 @@ import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
@@ -41,7 +43,7 @@ import static org.junit.Assert.assertEquals;
  * 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 the {@link Checkpointed} interface reflects the "exactly once" semantics.
+ * state defined with the {@link ListCheckpointed} interface reflects the "exactly once" semantics.
  */
 @SuppressWarnings("serial")
 public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
@@ -118,7 +120,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 	// --------------------------------------------------------------------------------------------
 	
 	private static class StringGeneratingSourceFunction extends RichSourceFunction<String>
-			implements ParallelSourceFunction<String>, Checkpointed<Integer> {
+			implements ParallelSourceFunction<String>, ListCheckpointed<Integer> {
 
 		private final long numElements;
 		
@@ -131,7 +133,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		private volatile boolean isRunning = true;
 
 		static final long[] counts = new long[PARALLELISM];
-		
+
 		@Override
 		public void close() throws IOException {
 			counts[getRuntimeContext().getIndexOfThisSubtask()] = index;
@@ -186,17 +188,20 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.index);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			index = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.index = state.get(0);
 		}
 	}
 	
-	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> implements Checkpointed<Long> {
+	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> implements ListCheckpointed<Long> {
 
 		private long count;
 		static final long[] counts = new long[PARALLELISM];
@@ -213,13 +218,16 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 
@@ -227,7 +235,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 	 * This function uses simultaneously the key/value state and is checkpointed.
 	 */
 	private static class OnceFailingPrefixCounter extends RichMapFunction<PrefixCount, PrefixCount> 
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 		
 		private static Map<String, Long> prefixCounts = new ConcurrentHashMap<String, Long>();
 		static final long[] counts = new long[PARALLELISM];
@@ -238,7 +246,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		
 		private long failurePos;
 		private long count;
-		
+
 		private ValueState<Long> pCount;
 		private long inputCount;
 
@@ -279,22 +287,25 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return inputCount;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.inputCount);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			inputCount = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.inputCount = state.get(0);
 		}
 	}
 
-	private static class StringRichFilterFunction extends RichFilterFunction<String> implements Checkpointed<Long> {
+	private static class StringRichFilterFunction extends RichFilterFunction<String> implements ListCheckpointed<Long> {
 		
 		static final long[] counts = new long[PARALLELISM];
 
 		private long count;
-		
+
 		@Override
 		public boolean filter(String value) {
 			count++;
@@ -307,23 +318,26 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 
 	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount>
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 		
 		static final long[] counts = new long[PARALLELISM];
 
 		private long count;
-		
+
 		@Override
 		public PrefixCount map(String value) throws IOException {
 			count++;
@@ -336,13 +350,16 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
index 5874f56..dae5cd9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -146,7 +145,7 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
 	 * augmented by the designated parallel subtaskId. The source is not parallel to ensure order.
 	 */
 	private static class StatefulMultipleSequence extends RichSourceFunction<Tuple2<Integer, Long>>
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 
 		private long count;
 
@@ -168,13 +167,16 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
 		public void cancel() {}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return count;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.count);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			count = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.count = state.get(0);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
index e424a8d..a45349d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
@@ -28,7 +28,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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.RichSourceFunction;
@@ -46,7 +46,9 @@ import org.junit.runners.Parameterized;
 
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
@@ -310,7 +312,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements Checkpointed<Integer>, CheckpointListener
+			implements ListCheckpointed<Integer>, CheckpointListener
 	{
 		private static volatile boolean failedBefore = false;
 
@@ -373,13 +375,16 @@ public class WindowCheckpointingITCase extends TestLogger {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsEmitted;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.numElementsEmitted);
 		}
 
 		@Override
-		public void restoreState(Integer state) {
-			numElementsEmitted = state;
+		public void restoreState(List<Integer> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.numElementsEmitted = state.get(0);
 		}
 
 		public static void reset() {
@@ -388,7 +393,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 	}
 
 	private static class ValidatingSink extends RichSinkFunction<Tuple2<Long, IntType>>
-			implements Checkpointed<HashMap<Long, Integer>> {
+			implements ListCheckpointed<HashMap<Long, Integer>> {
 
 		private final HashMap<Long, Integer> counts = new HashMap<>();
 
@@ -439,18 +444,20 @@ public class WindowCheckpointingITCase extends TestLogger {
 		}
 
 		@Override
-		public HashMap<Long, Integer> snapshotState(long checkpointId, long checkpointTimestamp) {
-			return this.counts;
+		public List<HashMap<Long, Integer>> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.counts);
 		}
 
 		@Override
-		public void restoreState(HashMap<Long, Integer> state) {
-			this.counts.putAll(state);
+		public void restoreState(List<HashMap<Long, Integer>> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.counts.putAll(state.get(0));
 
-			for (Integer i : state.values()) {
+			for (Integer i : state.get(0).values()) {
 				this.aggCount += i;
 			}
-
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
index f91582f..52a3ba8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
@@ -21,13 +21,15 @@ package org.apache.flink.test.classloading.jar;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 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.SourceFunction;
 
 import java.lang.RuntimeException;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * A simple streaming program, which is using the state checkpointing of Flink.
@@ -57,7 +59,7 @@ public class CheckpointedStreamingProgram {
 
 
 	// with Checkpoining
-	public static class SimpleStringGenerator implements SourceFunction<String>, Checkpointed<Integer> {
+	public static class SimpleStringGenerator implements SourceFunction<String>, ListCheckpointed<Integer> {
 		public boolean running = true;
 
 		@Override
@@ -74,32 +76,36 @@ public class CheckpointedStreamingProgram {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return null;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.emptyList();
 		}
 
 		@Override
-		public void restoreState(Integer state) {
+		public void restoreState(List<Integer> state) throws Exception {
 
 		}
 	}
 
-	public static class StatefulMapper implements MapFunction<String, String>, Checkpointed<StatefulMapper>, CheckpointListener {
+	public static class StatefulMapper implements MapFunction<String, String>, ListCheckpointed<StatefulMapper>, CheckpointListener {
 
 		private String someState;
 		private boolean atLeastOneSnapshotComplete = false;
 		private boolean restored = false;
 
 		@Override
-		public StatefulMapper snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return this;
+		public List<StatefulMapper> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this);
 		}
 
 		@Override
-		public void restoreState(StatefulMapper state) {
+		public void restoreState(List<StatefulMapper> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
 			restored = true;
-			this.someState = state.someState;
-			this.atLeastOneSnapshotComplete = state.atLeastOneSnapshotComplete;
+			StatefulMapper s = state.get(0);
+			this.someState = s.someState;
+			this.atLeastOneSnapshotComplete = s.atLeastOneSnapshotComplete;
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
index 6796cb0..d3baa7d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
@@ -34,7 +34,7 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
@@ -42,6 +42,8 @@ import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
 
 public class CheckpointingCustomKvStateProgram {
@@ -84,7 +86,7 @@ public class CheckpointingCustomKvStateProgram {
 		env.execute();
 	}
 
-	private static class InfiniteIntegerSource implements ParallelSourceFunction<Integer>, Checkpointed<Integer> {
+	private static class InfiniteIntegerSource implements ParallelSourceFunction<Integer>, ListCheckpointed<Integer> {
 		private static final long serialVersionUID = -7517574288730066280L;
 		private volatile boolean running = true;
 
@@ -104,17 +106,18 @@ public class CheckpointingCustomKvStateProgram {
 		}
 
 		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return 0;
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(0);
 		}
 
 		@Override
-		public void restoreState(Integer state) throws Exception {
+		public void restoreState(List<Integer> state) throws Exception {
 
 		}
 	}
 
-	private static class ReducingStateFlatMap extends RichFlatMapFunction<Tuple2<Integer, Integer>, Integer> implements Checkpointed<ReducingStateFlatMap>, CheckpointListener {
+	private static class ReducingStateFlatMap extends RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>
+			implements ListCheckpointed<ReducingStateFlatMap>, CheckpointListener {
 
 		private static final long serialVersionUID = -5939722892793950253L;
 		private transient ReducingState<Integer> kvState;
@@ -148,12 +151,12 @@ public class CheckpointingCustomKvStateProgram {
 		}
 
 		@Override
-		public ReducingStateFlatMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return this;
+		public List<ReducingStateFlatMap> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this);
 		}
 
 		@Override
-		public void restoreState(ReducingStateFlatMap state) throws Exception {
+		public void restoreState(List<ReducingStateFlatMap> state) throws Exception {
 			restored = true;
 			atLeastOneSnapshotComplete = true;
 		}


[14/17] flink git commit: [hotfix] [doc] Fix several broken "Linking with Flink" links

Posted by ch...@apache.org.
[hotfix] [doc] Fix several broken "Linking with Flink" links


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

Branch: refs/heads/release-1.2
Commit: 64c4c9ddeb0ad17352a81f8cb5c479e76097ea03
Parents: 8b069fd
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Tue Jan 17 12:21:29 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 23:59:11 2017 +0100

----------------------------------------------------------------------
 docs/dev/batch/index.md                | 2 +-
 docs/dev/connectors/cassandra.md       | 2 +-
 docs/dev/connectors/elasticsearch.md   | 2 +-
 docs/dev/connectors/elasticsearch2.md  | 4 ++--
 docs/dev/connectors/filesystem_sink.md | 2 +-
 docs/dev/connectors/kafka.md           | 2 +-
 docs/dev/connectors/kinesis.md         | 2 +-
 docs/dev/connectors/nifi.md            | 2 +-
 docs/dev/connectors/rabbitmq.md        | 2 +-
 docs/dev/connectors/twitter.md         | 2 +-
 docs/dev/libs/cep.md                   | 4 ++--
 docs/dev/libs/gelly/index.md           | 2 +-
 docs/dev/libs/ml/index.md              | 4 ++--
 docs/dev/libs/ml/quickstart.md         | 2 +-
 14 files changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/batch/index.md
----------------------------------------------------------------------
diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md
index 48d60e1..52807ca 100644
--- a/docs/dev/batch/index.md
+++ b/docs/dev/batch/index.md
@@ -49,7 +49,7 @@ Example Program
 
 The following program is a complete, working example of WordCount. You can copy &amp; paste the code
 to run it locally. You only have to include the correct Flink's library into your project
-(see Section [Linking with Flink]({{ site.baseurl }}/dev/linking_with_flink)) and specify the imports. Then you are ready
+(see Section [Linking with Flink]({{ site.baseurl }}/dev/linking_with_flink.html)) and specify the imports. Then you are ready
 to go!
 
 <div class="codetabs" markdown="1">

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/cassandra.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/cassandra.md b/docs/dev/connectors/cassandra.md
index 19d483b..7f76b72 100644
--- a/docs/dev/connectors/cassandra.md
+++ b/docs/dev/connectors/cassandra.md
@@ -35,7 +35,7 @@ To use this connector, add the following dependency to your project:
 </dependency>
 {% endhighlight %}
 
-Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/linking).
+Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/linking.html).
 
 #### Installing Apache Cassandra
 Follow the instructions from the [Cassandra Getting Started page](http://wiki.apache.org/cassandra/GettingStarted).

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/elasticsearch.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md
index 1907740..3e8c68a 100644
--- a/docs/dev/connectors/elasticsearch.md
+++ b/docs/dev/connectors/elasticsearch.md
@@ -37,7 +37,7 @@ following dependency to your project:
 
 Note that the streaming connectors are currently not part of the binary
 distribution. See
-[here]({{site.baseurl}}/dev/linking)
+[here]({{site.baseurl}}/dev/linking.html)
 for information about how to package the program with the libraries for
 cluster execution.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/elasticsearch2.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/elasticsearch2.md b/docs/dev/connectors/elasticsearch2.md
index a796280..af02c84 100644
--- a/docs/dev/connectors/elasticsearch2.md
+++ b/docs/dev/connectors/elasticsearch2.md
@@ -37,7 +37,7 @@ following dependency to your project:
 
 Note that the streaming connectors are currently not part of the binary
 distribution. See
-[here]({{site.baseurl}}/dev/linking)
+[here]({{site.baseurl}}/dev/linking.html)
 for information about how to package the program with the libraries for
 cluster execution.
 
@@ -145,7 +145,7 @@ More information about Elasticsearch can be found [here](https://elastic.co).
 
 For the execution of your Flink program,
 it is recommended to build a so-called uber-jar (executable jar) containing all your dependencies
-(see [here]({{site.baseurl}}/dev/linking) for further information).
+(see [here]({{site.baseurl}}/dev/linking.html) for further information).
 
 However,
 when an uber-jar containing an Elasticsearch sink is executed,

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/filesystem_sink.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/filesystem_sink.md b/docs/dev/connectors/filesystem_sink.md
index 030e9d9..0fa8bb1 100644
--- a/docs/dev/connectors/filesystem_sink.md
+++ b/docs/dev/connectors/filesystem_sink.md
@@ -37,7 +37,7 @@ following dependency to your project:
 
 Note that the streaming connectors are currently not part of the binary
 distribution. See
-[here]({{site.baseurl}}/dev/linking)
+[here]({{site.baseurl}}/dev/linking.html)
 for information about how to package the program with the libraries for
 cluster execution.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/kafka.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md
index e09befe..cc51071 100644
--- a/docs/dev/connectors/kafka.md
+++ b/docs/dev/connectors/kafka.md
@@ -82,7 +82,7 @@ Then, import the connector in your maven project:
 </dependency>
 {% endhighlight %}
 
-Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/linking).
+Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/linking.html).
 
 ### Installing Apache Kafka
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/kinesis.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/kinesis.md b/docs/dev/connectors/kinesis.md
index 480a97d..d95fe21 100644
--- a/docs/dev/connectors/kinesis.md
+++ b/docs/dev/connectors/kinesis.md
@@ -51,7 +51,7 @@ mvn clean install -Pinclude-kinesis -DskipTests
 
 
 The streaming connectors are not part of the binary distribution. See how to link with them for cluster
-execution [here]({{site.baseurl}}/dev/linking).
+execution [here]({{site.baseurl}}/dev/linking.html).
 
 ### Using the Amazon Kinesis Streams Service
 Follow the instructions from the [Amazon Kinesis Streams Developer Guide](https://docs.aws.amazon.com/streams/latest/dev/learning-kinesis-module-one-create-stream.html)

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/nifi.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/nifi.md b/docs/dev/connectors/nifi.md
index bdbd808..aa9eba2 100644
--- a/docs/dev/connectors/nifi.md
+++ b/docs/dev/connectors/nifi.md
@@ -37,7 +37,7 @@ following dependency to your project:
 
 Note that the streaming connectors are currently not part of the binary
 distribution. See
-[here]({{site.baseurl}}/dev/linking)
+[here]({{site.baseurl}}/dev/linking.html)
 for information about how to package the program with the libraries for
 cluster execution.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/rabbitmq.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/rabbitmq.md b/docs/dev/connectors/rabbitmq.md
index 7f117c6..47b5998 100644
--- a/docs/dev/connectors/rabbitmq.md
+++ b/docs/dev/connectors/rabbitmq.md
@@ -33,7 +33,7 @@ This connector provides access to data streams from [RabbitMQ](http://www.rabbit
 </dependency>
 {% endhighlight %}
 
-Note that the streaming connectors are currently not part of the binary distribution. See linking with them for cluster execution [here]({{site.baseurl}}/dev/linking).
+Note that the streaming connectors are currently not part of the binary distribution. See linking with them for cluster execution [here]({{site.baseurl}}/dev/linking.html).
 
 #### Installing RabbitMQ
 Follow the instructions from the [RabbitMQ download page](http://www.rabbitmq.com/download.html). After the installation the server automatically starts, and the application connecting to RabbitMQ can be launched.

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/connectors/twitter.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/twitter.md b/docs/dev/connectors/twitter.md
index 9b6a019..be15aaf 100644
--- a/docs/dev/connectors/twitter.md
+++ b/docs/dev/connectors/twitter.md
@@ -36,7 +36,7 @@ To use this connector, add the following dependency to your project:
 {% endhighlight %}
 
 Note that the streaming connectors are currently not part of the binary distribution.
-See linking with them for cluster execution [here]({{site.baseurl}}/dev/linking).
+See linking with them for cluster execution [here]({{site.baseurl}}/dev/linking.html).
 
 #### Authentication
 In order to connect to the Twitter stream the user has to register their program and acquire the necessary information for the authentication. The process is described below.

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index c30d37b..8047481 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -37,7 +37,7 @@ because these are used for comparing and matching events.
 
 ## Getting Started
 
-If you want to jump right in, you have to [set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink).
+If you want to jump right in, you have to [set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink.html).
 Next, you have to add the FlinkCEP dependency to the `pom.xml` of your project.
 
 <div class="codetabs" markdown="1">
@@ -63,7 +63,7 @@ Next, you have to add the FlinkCEP dependency to the `pom.xml` of your project.
 </div>
 
 Note that FlinkCEP is currently not part of the binary distribution.
-See linking with it for cluster execution [here]({{site.baseurl}}/dev/linking).
+See linking with it for cluster execution [here]({{site.baseurl}}/dev/linking.html).
 
 Now you can start writing your first CEP program using the pattern API.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/libs/gelly/index.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/gelly/index.md b/docs/dev/libs/gelly/index.md
index 6bcdc82..d82fec0 100644
--- a/docs/dev/libs/gelly/index.md
+++ b/docs/dev/libs/gelly/index.md
@@ -63,7 +63,7 @@ Add the following dependency to your `pom.xml` to use Gelly.
 </div>
 </div>
 
-Note that Gelly is currently not part of the binary distribution. See linking with it for cluster execution [here]({{ site.baseurl }}/dev/linking).
+Note that Gelly is currently not part of the binary distribution. See linking with it for cluster execution [here]({{ site.baseurl }}/dev/linking.html).
 
 The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink DataSet API.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/libs/ml/index.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/ml/index.md b/docs/dev/libs/ml/index.md
index dcd3e0a..b7d5285 100644
--- a/docs/dev/libs/ml/index.md
+++ b/docs/dev/libs/ml/index.md
@@ -68,7 +68,7 @@ FlinkML currently supports the following algorithms:
 You can check out our [quickstart guide](quickstart.html) for a comprehensive getting started
 example.
 
-If you want to jump right in, you have to [set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink).
+If you want to jump right in, you have to [set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink.html).
 Next, you have to add the FlinkML dependency to the `pom.xml` of your project.
 
 {% highlight xml %}
@@ -80,7 +80,7 @@ Next, you have to add the FlinkML dependency to the `pom.xml` of your project.
 {% endhighlight %}
 
 Note that FlinkML is currently not part of the binary distribution.
-See linking with it for cluster execution [here]({{site.baseurl}}/dev/linking).
+See linking with it for cluster execution [here]({{site.baseurl}}/dev/linking.html).
 
 Now you can start solving your analysis task.
 The following code snippet shows how easy it is to train a multiple linear regression model.

http://git-wip-us.apache.org/repos/asf/flink/blob/64c4c9dd/docs/dev/libs/ml/quickstart.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/ml/quickstart.md b/docs/dev/libs/ml/quickstart.md
index 29f2fec..5dff6bb 100644
--- a/docs/dev/libs/ml/quickstart.md
+++ b/docs/dev/libs/ml/quickstart.md
@@ -55,7 +55,7 @@ through [principal components analysis](https://en.wikipedia.org/wiki/Principal_
 ## Linking with FlinkML
 
 In order to use FlinkML in your project, first you have to
-[set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink).
+[set up a Flink program]({{ site.baseurl }}/dev/linking_with_flink.html).
 Next, you have to add the FlinkML dependency to the `pom.xml` of your project:
 
 {% highlight xml %}


[05/17] flink git commit: [FLINK-5434] Remove unsupported project() transformation from Scala DataStream docs.

Posted by ch...@apache.org.
[FLINK-5434] Remove unsupported project() transformation from Scala DataStream docs.


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

Branch: refs/heads/release-1.2
Commit: ebf4443a87ec0a46cd821adc85187d9cb58ae992
Parents: 8cdbe44
Author: gaolun.gl <ga...@alibaba-inc.com>
Authored: Sat Jan 14 01:41:39 2017 +0800
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:00 2017 +0100

----------------------------------------------------------------------
 docs/dev/datastream_api.md | 28 ----------------------------
 1 file changed, 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ebf4443a/docs/dev/datastream_api.md
----------------------------------------------------------------------
diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md
index 850d8c5..f6d17ab 100644
--- a/docs/dev/datastream_api.md
+++ b/docs/dev/datastream_api.md
@@ -911,34 +911,6 @@ DataStream<Tuple2<String, Integer>> out = in.project(2,0);
 </table>
 
 </div>
-
-<div data-lang="scala" markdown="1">
-
-<br />
-
-<table class="table table-bordered">
-  <thead>
-    <tr>
-      <th class="text-left" style="width: 20%">Transformation</th>
-      <th class="text-center">Description</th>
-    </tr>
-  </thead>
-  <tbody>
-   <tr>
-      <td><strong>Project</strong><br>DataStream &rarr; DataStream</td>
-      <td>
-        <p>Selects a subset of fields from the tuples
-{% highlight scala %}
-val in : DataStream[(Int,Double,String)] = // [...]
-val out = in.project(2,0)
-{% endhighlight %}
-        </p>
-      </td>
-    </tr>
-  </tbody>
-</table>
-
-</div>
 </div>
 
 


[17/17] flink git commit: Rebuild web-dashboard

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


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

Branch: refs/heads/release-1.2
Commit: 080617d28741fa087cdf97a2d50618c195a85a11
Parents: 82a6b83
Author: zentol <ch...@apache.org>
Authored: Thu Jan 19 23:54:08 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Fri Jan 20 00:07:21 2017 +0100

----------------------------------------------------------------------
 flink-runtime-web/web-dashboard/web/css/index.css              | 2 +-
 flink-runtime-web/web-dashboard/web/js/index.js                | 4 ++--
 .../web/partials/jobs/job.plan.node-list.metrics.html          | 6 +++---
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/080617d2/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
index 3129459..e2fa6c1 100644
--- a/flink-runtime-web/web-dashboard/web/css/index.css
+++ b/flink-runtime-web/web-dashboard/web/css/index.css
@@ -1 +1 @@
-#main,#sidebar,body,html{height:100%}#content,#sidebar{-webkit-transition:.4s;-moz-transition:.4s;-o-transition:.4s;-ms-transition:.4s}.gutter{background-color:transparent;background-repeat:no-repeat;background-position:50%}.gutter-vertical{cursor:row-resize;background-image:url(/images/grips/horizontal.png)}#sidebar{overflow:hidden;position:fixed;left:-250px;top:0;bottom:0;width:250px;background:#151515;transition:.4s;-webkit-box-shadow:inset -10px 0 10px rgba(0,0,0,.2);box-shadow:inset -10px 0 10px rgba(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:700;color:#fff;padding-left:0}#sidebar .nav>li>a{color:#aaa;margin-bottom:1px}#sidebar .nav>li>a:focus,#sidebar .nav>li>a:hover{background-color:rgba(40,40,40,.5)}#sidebar .nav>li>a.active
 {background-color:rgba(100,100,100,.5)}#content{background-color:#fff;margin-left:0;padding-top:70px;height:100%;transition:.4s}.table .table,.table.table-inner{background-color:transparent}#content .navbar-main,#content .navbar-main-additional{-webkit-transition:.4s;-moz-transition:.4s;-o-transition:.4s;-ms-transition:.4s;transition:.4s}#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:0 20px 20px}#content #content-inner.has-navbar-main-additional{padding-top:42px}.table#add-file-table span.btn,.table#job-su
 bmit-table td>span.btn{padding:2px 4px;font-size:14px}.page-header{margin:0 0 20px}.nav>li>a,.nav>li>a:focus,.nav>li>a:hover{color:#aaa;background-color:transparent;border-bottom:2px solid transparent}.nav>li.active>a,.nav>li.active>a:focus,.nav>li.active>a:hover{color:#000;border-bottom:2px solid #000}.nav.nav-tabs{margin-bottom:20px}.table th{font-weight:400;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-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.active td.tab-column li.active,.table.table-body-hover>tbody.active td:not(.tab-col
 umn),.table.table-body-hover>tbody:hover td.tab-column li.active,.table.table-body-hover>tbody:hover td:not(.tab-column){background-color:#f0f0f0}.table.table-activable td.tab-column,.table.table-activable th.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.table-no-border td,.table.table-no-border th{border-top:none!important}.table#job-submit-table{table-layout:fixed;white-space:nowrap}.table#job-submit-table td.td-large{width:40%}.table#job-submit-table td{width:15%}.table#job-submit-table td>input{height:28px;font-size:14px}.table#add-file-table{table-layout:fixed}.table#add-file-table span.btn{position:relative;overflow:hidden;border-radius:2px;margin-top:-3px}.table#add-file-table td#add-file-button{width:100px}.table#add-file-table td#add-file-button input[type=file]{position:absolute;top:0;right:0;min-width:100%;min-height:100%;opacity:0;-ms-filter:"progid:DXImageTransfor
 m.Microsoft.Alpha(Opacity=0)";filter:alpha(opacity=0);outline:0;cursor:inherit;display:block}.timeline-canvas .timeline-insidelabel,.timeline-canvas .timeline-series,svg.graph .node{cursor:pointer}.table#add-file-table td#add-file-name{width:250px;-o-text-overflow:ellipsis;text-overflow:ellipsis;overflow:hidden;white-space:nowrap}.table#add-file-table td#add-file-status{width:100%}.table#add-file-table td#add-file-status span.btn-progress-bar{padding:0!important;width:100%;background-color:#f5f5f5;text-align:left}.table#add-file-table td#add-file-status span.btn-progress{padding:2px;font-size:10px}.table span.error-area{color:red}.table span.row-button{padding:1px 2px;margin:0;border:none!important}.table .small-label{text-transform:uppercase;font-size:13px;color:#999}span.icon-wrapper{width:1.2em;display:inline-block}.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-t
 itle,.navbar-main .panel-title,.navbar-main-additional .navbar-title,.navbar-main-additional .panel-title,.navbar-secondary .navbar-title,.navbar-secondary .panel-title,.navbar-secondary-additional .navbar-title,.navbar-secondary-additional .panel-title,.panel.panel-multi .navbar-title,.panel.panel-multi .panel-title{float:left;font-size:18px;padding:12px 20px 13px 10px;color:#333;display:inline-block}.navbar-main .navbar-info,.navbar-main .panel-info,.navbar-main-additional .navbar-info,.navbar-main-additional .panel-info,.navbar-secondary .navbar-info,.navbar-secondary .panel-info,.navbar-secondary-additional .navbar-info,.navbar-secondary-additional .panel-info,.panel.panel-multi .navbar-info,.panel.panel-multi .panel-info{float:left;font-size:14px;padding:15px;color:#999;display:inline-block;border-right:1px solid #e7e7e7;overflow:hidden}.navbar-main .navbar-info .overflow,.navbar-main .panel-info .overflow,.navbar-main-additional .navbar-info .overflow,.navbar-main-additional .
 panel-info .overflow,.navbar-secondary .navbar-info .overflow,.navbar-secondary .panel-info .overflow,.navbar-secondary-additional .navbar-info .overflow,.navbar-secondary-additional .panel-info .overflow,.panel.panel-multi .navbar-info .overflow,.panel.panel-multi .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-main .panel-info.first,.navbar-main-additional .navbar-info.first,.navbar-main-additional .panel-info.first,.navbar-secondary .navbar-info.first,.navbar-secondary .panel-info.first,.navbar-secondary-additional .navbar-info.first,.navbar-secondary-additional .panel-info.first,.panel.panel-multi .navbar-info.first,.panel.panel-multi .panel-info.first{border-left:1px solid #e7e7e7}.navbar-main .navbar-info.last,.navbar-main .panel-info.last,.navbar-main-additional .navbar-info.last,.navbar-main-additional .panel-info.l
 ast,.navbar-secondary .navbar-info.last,.navbar-secondary .panel-info.last,.navbar-secondary-additional .navbar-info.last,.navbar-secondary-additional .panel-info.last,.panel.panel-multi .navbar-info.last,.panel.panel-multi .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}.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]:focus,.badge-default[href]:hover{background-color:grey}.badge-primary{background-color:#
 428bca}.badge-primary[href]:focus,.badge-primary[href]:hover{background-color:#3071a9}.badge-success{background-color:#5cb85c}.badge-success[href]:focus,.badge-success[href]:hover{background-color:#449d44}.badge-info{background-color:#5bc0de}.badge-info[href]:focus,.badge-info[href]:hover{background-color:#31b0d5}.badge-warning{background-color:#f0ad4e}.badge-warning[href]:focus,.badge-warning[href]:hover{background-color:#ec971f}.badge-danger{background-color:#d9534f}.badge-danger[href]:focus,.badge-danger[href]:hover{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}.checkpoint-overview a,svg.graph .node h4{color:#000}pre.exception{border:none;background-color:transparent;padding:0;margin:0}pre{white-space:pre-wrap;white-space:-moz-pre-wrap;white-sp
 ace:-pre-wrap;white-space:-o-pre-wrap;word-wrap:break-word}.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;border-radius:0;border-bottom:none;border-left:2px solid transparent}.nav-tabs.tabs-vertical li.active>a,.nav-tabs.tabs-vertical li>a:focus,.nav-tabs.tabs-vertical li>a:hover{border-bottom:none;border-left:2px solid #000}.navbar-main .navbar-title,.navbar-main-additional .navbar-title,.navbar-secondary .navbar-title,.navbar-secondary-additional .navbar-title{padding:12px 20px 13px}livechart{width:30%;height:30%;text-align:center}.canvas-wrapper{border:1px solid #ddd;position:relative;margin-bottom:20px;height:100%}.canvas-wrapper .main-canvas{height:100%;overflow:hidden}.canvas-wrapper .main-canvas .zoom-buttons{position:absolute;top:10px;right:10px}.label-group .label{display:inline-block;padding-left:.4em;padding-right:.4em;
 margin:0;border-right:1px solid #fff;border-radius:0}.label-group .label.label-black{background-color:#000}.navbar-info-button{padding:3px 4px;font-size:12px;font-family:inherit;margin-top:-2px}svg.graph .edge-label,svg.graph text{font-size:14px}.checkpoints-view{padding-top:1em}.subtask-details .blank{height:2em}.checkpoint-overview td span{padding-left:2em}svg.graph{overflow:hidden;height:100%}svg.graph g.type-TK>rect{fill:#00ffd0}svg.graph text{font-weight:300}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 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 .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.secondary .timeline-insidelabel,.timeline-canvas.secondary .timeline-series{cursor:auto}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics a,.show-pointer{cursor:pointer}.qtip-timeline-bar{font-size:14px;line-height:1.4}#content .navbar-secondary-additional.navbar-secondary-additional-2{margin:-10px -10px 10px;padding:0;border-bottom:1px solid #e4e4e4}#content .navbar-secondary-additional.navbar-secondary-additional-2 .navbar-info{padding-top:12px;padding-bottom:12px}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics{margin-right:15px;float:right}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics .btn{margin-top:5px;margin-bottom:5px}#content .navbar-secondary-additional.navbar-secondary-additional
 -2 .metric-menu{max-height:300px;overflow-y:scroll}.metric-row{margin:0;min-height:275px;padding:0;list-style-type:none}.metric-row .metric-col{background-color:transparent;width:33.33%;float:left}.metric-row .metric-col.big{width:100%}.metric-row .metric-col .panel{margin-left:5px;margin-right:5px;min-height:265px;margin-bottom:10px}.metric-row .metric-col .panel .panel-body{background-color:transparent;height:265px;position:relative}.metric-row .metric-col .panel .panel-heading{padding:0 10px;background-color:transparent;height:41px;line-height:41px;position:relative;overflow:hidden;cursor:pointer}.metric-row .metric-col .panel .panel-heading .metric-title{padding:10px 0}.metric-row .metric-col .panel .panel-heading .buttons{position:absolute;top:0;right:0;padding:0 10px;background-color:#fff}.metric-row .metric-col.dndDraggingSource{display:none}.metric-row .dndPlaceholder{position:relative;background-color:#f0f0f0;min-height:305px;display:block;width:33.33%;float:left;margin-bot
 tom:10px;border-radius:5px}.p-info{padding-left:5px;padding-right:5px}@media (min-width:1024px) and (max-width:1279px){#content #fold-button,#sidebar .navbar-static-top .navbar-brand-text{display:none}#sidebar{left:0;width:160px}#content{margin-left:160px}#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%}}.legend-box{font-size:10px;width:2em}#total-mem{background-color:#7cb5ec}#heap-mem{background-color:#434348}#non-heap-mem{background-color:#90ed7d}#fetch-plan,#job-submit{width:100px}#content-inner,#details,#node-details{height:100%}#job-panel{overflow-y:auto}
\ No newline at end of file
+#main,#sidebar,body,html{height:100%}#content,#sidebar{-webkit-transition:.4s;-moz-transition:.4s;-o-transition:.4s;-ms-transition:.4s}.gutter{background-color:transparent;background-repeat:no-repeat;background-position:50%}.gutter-vertical{cursor:row-resize;background-image:url(/images/grips/horizontal.png)}#sidebar{overflow:hidden;position:fixed;left:-250px;top:0;bottom:0;width:250px;background:#151515;transition:.4s;-webkit-box-shadow:inset -10px 0 10px rgba(0,0,0,.2);box-shadow:inset -10px 0 10px rgba(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:700;color:#fff;padding-left:0}#sidebar .nav>li>a{color:#aaa;margin-bottom:1px}#sidebar .nav>li>a:focus,#sidebar .nav>li>a:hover{background-color:rgba(40,40,40,.5)}#sidebar .nav>li>a.active
 {background-color:rgba(100,100,100,.5)}#content{background-color:#fff;margin-left:0;padding-top:70px;height:100%;transition:.4s}.table .table,.table.table-inner{background-color:transparent}#content .navbar-main,#content .navbar-main-additional{-webkit-transition:.4s;-moz-transition:.4s;-o-transition:.4s;-ms-transition:.4s;transition:.4s}#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:0 20px 20px}#content #content-inner.has-navbar-main-additional{padding-top:42px}.table#add-file-table span.btn,.table#job-su
 bmit-table td>span.btn{padding:2px 4px;font-size:14px}.page-header{margin:0 0 20px}.nav>li>a,.nav>li>a:focus,.nav>li>a:hover{color:#aaa;background-color:transparent;border-bottom:2px solid transparent}.nav>li.active>a,.nav>li.active>a:focus,.nav>li.active>a:hover{color:#000;border-bottom:2px solid #000}.nav.nav-tabs{margin-bottom:20px}.table th{font-weight:400;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-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.active td.tab-column li.active,.table.table-body-hover>tbody.active td:not(.tab-col
 umn),.table.table-body-hover>tbody:hover td.tab-column li.active,.table.table-body-hover>tbody:hover td:not(.tab-column){background-color:#f0f0f0}.table.table-activable td.tab-column,.table.table-activable th.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.table-no-border td,.table.table-no-border th{border-top:none!important}.table#job-submit-table{table-layout:fixed;white-space:nowrap}.table#job-submit-table td.td-large{width:40%}.table#job-submit-table td{width:15%}.table#job-submit-table td>input{height:28px;font-size:14px}.table#add-file-table{table-layout:fixed}.table#add-file-table span.btn{position:relative;overflow:hidden;-webkit-border-radius:2px;border-radius:2px;margin-top:-3px}.table#add-file-table td#add-file-button{width:100px}.table#add-file-table td#add-file-button input[type=file]{position:absolute;top:0;right:0;min-width:100%;min-height:100%;opacity:0;-ms-filt
 er:"progid:DXImageTransform.Microsoft.Alpha(Opacity=0)";filter:alpha(opacity=0);outline:0;cursor:inherit;display:block}.timeline-canvas .timeline-insidelabel,.timeline-canvas .timeline-series,svg.graph .node{cursor:pointer}.table#add-file-table td#add-file-name{width:250px;-o-text-overflow:ellipsis;text-overflow:ellipsis;overflow:hidden;white-space:nowrap}.table#add-file-table td#add-file-status{width:100%}.table#add-file-table td#add-file-status span.btn-progress-bar{padding:0!important;width:100%;background-color:#f5f5f5;text-align:left}.table#add-file-table td#add-file-status span.btn-progress{padding:2px;font-size:10px}.table span.error-area{color:red}.table span.row-button{padding:1px 2px;margin:0;border:none!important}.table .small-label{text-transform:uppercase;font-size:13px;color:#999}span.icon-wrapper{width:1.2em;display:inline-block}.panel.panel-dashboard .huge{font-size:28px}.panel.panel-lg{font-size:16px}.panel.panel-lg .badge{font-size:14px}.navbar-secondary{overflow:a
 uto}.navbar-main .navbar-title,.navbar-main .panel-title,.navbar-main-additional .navbar-title,.navbar-main-additional .panel-title,.navbar-secondary .navbar-title,.navbar-secondary .panel-title,.navbar-secondary-additional .navbar-title,.navbar-secondary-additional .panel-title,.panel.panel-multi .navbar-title,.panel.panel-multi .panel-title{float:left;font-size:18px;padding:12px 20px 13px 10px;color:#333;display:inline-block}.navbar-main .navbar-info,.navbar-main .panel-info,.navbar-main-additional .navbar-info,.navbar-main-additional .panel-info,.navbar-secondary .navbar-info,.navbar-secondary .panel-info,.navbar-secondary-additional .navbar-info,.navbar-secondary-additional .panel-info,.panel.panel-multi .navbar-info,.panel.panel-multi .panel-info{float:left;font-size:14px;padding:15px;color:#999;display:inline-block;border-right:1px solid #e7e7e7;overflow:hidden}.navbar-main .navbar-info .overflow,.navbar-main .panel-info .overflow,.navbar-main-additional .navbar-info .overflow
 ,.navbar-main-additional .panel-info .overflow,.navbar-secondary .navbar-info .overflow,.navbar-secondary .panel-info .overflow,.navbar-secondary-additional .navbar-info .overflow,.navbar-secondary-additional .panel-info .overflow,.panel.panel-multi .navbar-info .overflow,.panel.panel-multi .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-main .panel-info.first,.navbar-main-additional .navbar-info.first,.navbar-main-additional .panel-info.first,.navbar-secondary .navbar-info.first,.navbar-secondary .panel-info.first,.navbar-secondary-additional .navbar-info.first,.navbar-secondary-additional .panel-info.first,.panel.panel-multi .navbar-info.first,.panel.panel-multi .panel-info.first{border-left:1px solid #e7e7e7}.navbar-main .navbar-info.last,.navbar-main .panel-info.last,.navbar-main-additional .navbar-info.last,.navbar-mai
 n-additional .panel-info.last,.navbar-secondary .navbar-info.last,.navbar-secondary .panel-info.last,.navbar-secondary-additional .navbar-info.last,.navbar-secondary-additional .panel-info.last,.panel.panel-multi .navbar-info.last,.panel.panel-multi .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}.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]:focus,.badge-default[href]:hover{background-color:grey}.badge-
 primary{background-color:#428bca}.badge-primary[href]:focus,.badge-primary[href]:hover{background-color:#3071a9}.badge-success{background-color:#5cb85c}.badge-success[href]:focus,.badge-success[href]:hover{background-color:#449d44}.badge-info{background-color:#5bc0de}.badge-info[href]:focus,.badge-info[href]:hover{background-color:#31b0d5}.badge-warning{background-color:#f0ad4e}.badge-warning[href]:focus,.badge-warning[href]:hover{background-color:#ec971f}.badge-danger{background-color:#d9534f}.badge-danger[href]:focus,.badge-danger[href]:hover{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}.checkpoint-overview a,svg.graph .node h4{color:#000}pre.exception{border:none;background-color:transparent;padding:0;margin:0}pre{white-space:pre-wrap;white-sp
 ace:-moz-pre-wrap;white-space:-pre-wrap;white-space:-o-pre-wrap;word-wrap:break-word}.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.active>a,.nav-tabs.tabs-vertical li>a:focus,.nav-tabs.tabs-vertical li>a:hover{border-bottom:none;border-left:2px solid #000}.navbar-main .navbar-title,.navbar-main-additional .navbar-title,.navbar-secondary .navbar-title,.navbar-secondary-additional .navbar-title{padding:12px 20px 13px}livechart{width:30%;height:30%;text-align:center}.canvas-wrapper{border:1px solid #ddd;position:relative;margin-bottom:20px;height:100%}.canvas-wrapper .main-canvas{height:100%;overflow:hidden}.canvas-wrapper .main-canvas .zoom-buttons{position:absolute;top:10px;right:10px}.label-group .label{display:
 inline-block;padding-left:.4em;padding-right:.4em;margin:0;border-right:1px solid #fff;-webkit-border-radius:0;border-radius:0}.label-group .label.label-black{background-color:#000}.navbar-info-button{padding:3px 4px;font-size:12px;font-family:inherit;margin-top:-2px}svg.graph .edge-label,svg.graph text{font-size:14px}.checkpoints-view{padding-top:1em}.subtask-details .blank{height:2em}.checkpoint-overview td span{padding-left:2em}svg.graph{overflow:hidden;height:100%}svg.graph g.type-TK>rect{fill:#00ffd0}svg.graph text{font-weight:300}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 h5{color:#999}svg.graph .edgeLabel rect{fill:#fff}svg.graph .edgePath path{strok
 e:#333;stroke-width:2px;fill:#333}svg.graph .label{color:#777;margin:0}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.secondary .timeline-insidelabel,.timeline-canvas.secondary .timeline-series{cursor:auto}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics a,.show-pointer{cursor:pointer}.qtip-timeline-bar{font-size:14px;line-height:1.4}#content .navbar-secondary-additional.navbar-secondary-additional-2{margin:-10px -10px 10px;padding:0;border-bottom:1px solid #e4e4e4}#content .navbar-secondary-additional.navbar-secondary-additional-2 .navbar-info{padding-top:12px;padding-bottom:12px}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics{margin-right:15px;float:right}#content .navbar-secondary-additional.navbar-secondary-additional-2 .add-metrics .btn{margin-top:5px;margin-bo
 ttom:5px}#content .navbar-secondary-additional.navbar-secondary-additional-2 .metric-menu{max-height:300px;max-width:900px;overflow-y:scroll;text-align:right}.metric-row{margin:0;min-height:275px;padding:0;list-style-type:none}.metric-row .metric-col{background-color:transparent;width:33.33%;float:left}.metric-row .metric-col.big{width:100%}.metric-row .metric-col .panel{margin-left:5px;margin-right:5px;min-height:265px;margin-bottom:10px}.metric-row .metric-col .panel .panel-body{background-color:transparent;height:265px;position:relative}.metric-row .metric-col .panel .panel-heading{padding:0 10px;background-color:transparent;height:41px;line-height:41px;position:relative;overflow:hidden;cursor:pointer}.metric-row .metric-col .panel .panel-heading .metric-title{padding:10px 0}.metric-row .metric-col .panel .panel-heading .buttons{position:absolute;top:0;right:0;padding:0 10px;background-color:#fff}.metric-row .metric-col.dndDraggingSource{display:none}.metric-row .dndPlaceholder{p
 osition:relative;background-color:#f0f0f0;min-height:305px;display:block;width:33.33%;float:left;margin-bottom:10px;-webkit-border-radius:5px;border-radius:5px}.p-info{padding-left:5px;padding-right:5px}@media (min-width:1024px) and (max-width:1279px){#content #fold-button,#sidebar .navbar-static-top .navbar-brand-text{display:none}#sidebar{left:0;width:160px}#content{margin-left:160px}#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%}}.legend-box{font-size:10px;width:2em}#total-mem{background-color:#7cb5ec}#heap-mem{background-color:#434348}#non-heap-mem{background-color:#90ed7d}#fetch-plan,#job-submit{width:100px}#content-inner,#details,#node-details{height:100%}#job-panel{overflow-y:auto}
\ No newline at end of file


[02/17] flink git commit: [FLINK-5432] recursively scan nested files in ContinuousFileMonitoringFunction

Posted by ch...@apache.org.
[FLINK-5432] recursively scan nested files in ContinuousFileMonitoringFunction


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

Branch: refs/heads/release-1.2
Commit: 28c18e22127a85f773e7504a0e9d188bad9334e2
Parents: 79b6826
Author: Yassine Marzougui <y....@mindlytix.com>
Authored: Wed Jan 11 01:43:19 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:41:59 2017 +0100

----------------------------------------------------------------------
 .../flink/api/common/io/FileInputFormat.java    |  2 +-
 .../hdfstests/ContinuousFileProcessingTest.java | 53 ++++++++++++++++++++
 .../ContinuousFileMonitoringFunction.java       | 18 ++++---
 3 files changed, 65 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/28c18e22/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
index 1d092af..785fb3b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
@@ -637,7 +637,7 @@ public abstract class FileInputFormat<OT> extends RichInputFormat<OT, FileInputS
 	 * @param fileStatus The file status to check.
 	 * @return true, if the given file or directory is accepted
 	 */
-	protected boolean acceptFile(FileStatus fileStatus) {
+	public boolean acceptFile(FileStatus fileStatus) {
 		final String name = fileStatus.getPath().getName();
 		return !name.startsWith("_")
 			&& !name.startsWith(".")

http://git-wip-us.apache.org/repos/asf/flink/blob/28c18e22/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java
index 0cb1bad..c29dd27 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java
@@ -588,6 +588,59 @@ public class ContinuousFileProcessingTest {
 	}
 
 	@Test
+	public void testNestedFilesProcessing() throws Exception {
+		final Set<org.apache.hadoop.fs.Path> filesCreated = new HashSet<>();
+		final Set<String> filesToBeRead = new TreeSet<>();
+
+		// create two nested directories
+		org.apache.hadoop.fs.Path firstLevelDir = new org.apache.hadoop.fs.Path(hdfsURI + "/" + "firstLevelDir");
+		org.apache.hadoop.fs.Path secondLevelDir = new org.apache.hadoop.fs.Path(hdfsURI + "/" + "firstLevelDir" + "/" + "secondLevelDir");
+		Assert.assertFalse(hdfs.exists(firstLevelDir));
+		hdfs.mkdirs(firstLevelDir);
+		hdfs.mkdirs(secondLevelDir);
+
+		// create files in the base dir, the first level dir and the second level dir
+		for (int i = 0; i < NO_OF_FILES; i++) {
+			Tuple2<org.apache.hadoop.fs.Path, String> file = createFileAndFillWithData(hdfsURI, "firstLevelFile", i, "This is test line.");
+			filesCreated.add(file.f0);
+			filesToBeRead.add(file.f0.getName());
+		}
+		for (int i = 0; i < NO_OF_FILES; i++) {
+			Tuple2<org.apache.hadoop.fs.Path, String> file = createFileAndFillWithData(firstLevelDir.toString(), "secondLevelFile", i, "This is test line.");
+			filesCreated.add(file.f0);
+			filesToBeRead.add(file.f0.getName());
+		}
+		for (int i = 0; i < NO_OF_FILES; i++) {
+			Tuple2<org.apache.hadoop.fs.Path, String> file = createFileAndFillWithData(secondLevelDir.toString(), "thirdLevelFile", i, "This is test line.");
+			filesCreated.add(file.f0);
+			filesToBeRead.add(file.f0.getName());
+		}
+
+		TextInputFormat format = new TextInputFormat(new Path(hdfsURI));
+		format.setFilesFilter(FilePathFilter.createDefaultFilter());
+		format.setNestedFileEnumeration(true);
+
+		ContinuousFileMonitoringFunction<String> monitoringFunction =
+			new ContinuousFileMonitoringFunction<>(format,
+				FileProcessingMode.PROCESS_ONCE, 1, INTERVAL);
+
+		final FileVerifyingSourceContext context =
+			new FileVerifyingSourceContext(new OneShotLatch(), monitoringFunction);
+
+		monitoringFunction.open(new Configuration());
+		monitoringFunction.run(context);
+
+		Assert.assertArrayEquals(filesToBeRead.toArray(), context.getSeenFiles().toArray());
+
+		// finally delete the dirs and the files created for the test.
+		for (org.apache.hadoop.fs.Path file: filesCreated) {
+			hdfs.delete(file, false);
+		}
+		hdfs.delete(secondLevelDir, false);
+		hdfs.delete(firstLevelDir, false);
+	}
+
+	@Test
 	public void testSortingOnModTime() throws Exception {
 		final long[] modTimes = new long[NO_OF_FILES];
 		final org.apache.hadoop.fs.Path[] filesCreated = new org.apache.hadoop.fs.Path[NO_OF_FILES];

http://git-wip-us.apache.org/repos/asf/flink/blob/28c18e22/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
index e0a042a..589e285 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
@@ -232,7 +232,7 @@ public class ContinuousFileMonitoringFunction<OUT>
 											SourceContext<TimestampedFileInputSplit> context) throws IOException {
 		assert (Thread.holdsLock(checkpointLock));
 
-		Map<Path, FileStatus> eligibleFiles = listEligibleFiles(fs);
+		Map<Path, FileStatus> eligibleFiles = listEligibleFiles(fs, new Path(path));
 		Map<Long, List<TimestampedFileInputSplit>> splitsSortedByModTime = getInputSplitsSortedByModTime(eligibleFiles);
 
 		for (Map.Entry<Long, List<TimestampedFileInputSplit>> splits: splitsSortedByModTime.entrySet()) {
@@ -282,11 +282,11 @@ public class ContinuousFileMonitoringFunction<OUT>
 	 * Returns the paths of the files not yet processed.
 	 * @param fileSystem The filesystem where the monitored directory resides.
 	 */
-	private Map<Path, FileStatus> listEligibleFiles(FileSystem fileSystem) throws IOException {
+	private Map<Path, FileStatus> listEligibleFiles(FileSystem fileSystem, Path path) throws IOException {
 
 		final FileStatus[] statuses;
 		try {
-			statuses = fileSystem.listStatus(new Path(path));
+			statuses = fileSystem.listStatus(path);
 		} catch (IOException e) {
 			// we may run into an IOException if files are moved while listing their status
 			// delay the check for eligible files in this case
@@ -300,10 +300,14 @@ public class ContinuousFileMonitoringFunction<OUT>
 			Map<Path, FileStatus> files = new HashMap<>();
 			// handle the new files
 			for (FileStatus status : statuses) {
-				Path filePath = status.getPath();
-				long modificationTime = status.getModificationTime();
-				if (!shouldIgnore(filePath, modificationTime)) {
-					files.put(filePath, status);
+				if (!status.isDir()) {
+					Path filePath = status.getPath();
+					long modificationTime = status.getModificationTime();
+					if (!shouldIgnore(filePath, modificationTime)) {
+						files.put(filePath, status);
+					}
+				} else if (format.getNestedFileEnumeration() && format.acceptFile(status)){
+					files.putAll(listEligibleFiles(fileSystem, status.getPath()));
 				}
 			}
 			return files;


[12/17] flink git commit: [FLINK-5113] Port functions in tests to new CheckpointedFunction IF.

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
new file mode 100644
index 0000000..60253fa
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.classloading.jar;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.runtime.state.CheckpointListener;
+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.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+
+/**
+ * This test is the same as the {@link CheckpointedStreamingProgram} but using the
+ * old and deprecated {@link Checkpointed} interface. It stays here in order to
+ * guarantee that although deprecated, the old Checkpointed interface is still supported.
+ * This is necessary to not break user code.
+ * */
+public class LegacyCheckpointedStreamingProgram {
+
+	private static final int CHECKPOINT_INTERVALL = 100;
+
+	public static void main(String[] args) throws Exception {
+		final String jarFile = args[0];
+		final String host = args[1];
+		final int port = Integer.parseInt(args[2]);
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		env.getConfig().disableSysoutLogging();
+		env.enableCheckpointing(CHECKPOINT_INTERVALL);
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000));
+		env.disableOperatorChaining();
+
+		DataStream<String> text = env.addSource(new SimpleStringGenerator());
+		text.map(new StatefulMapper()).addSink(new NoOpSink());
+		env.setParallelism(1);
+		env.execute("Checkpointed Streaming Program");
+	}
+
+
+	// with Checkpointing
+	public static class SimpleStringGenerator implements SourceFunction<String>, Checkpointed<Integer> {
+
+		private static final long serialVersionUID = 3700033137820808611L;
+
+		public boolean running = true;
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			while(running) {
+				Thread.sleep(1);
+				ctx.collect("someString");
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return null;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+
+		}
+	}
+
+	public static class StatefulMapper implements MapFunction<String, String>, Checkpointed<StatefulMapper>, CheckpointListener {
+
+		private static final long serialVersionUID = 2703630582894634440L;
+
+		private String someState;
+		private boolean atLeastOneSnapshotComplete = false;
+		private boolean restored = false;
+
+		@Override
+		public StatefulMapper snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return this;
+		}
+
+		@Override
+		public void restoreState(StatefulMapper state) {
+			restored = true;
+			this.someState = state.someState;
+			this.atLeastOneSnapshotComplete = state.atLeastOneSnapshotComplete;
+		}
+
+		@Override
+		public String map(String value) throws Exception {
+			if(!atLeastOneSnapshotComplete) {
+				// throttle consumption by the checkpoint interval until we have one snapshot.
+				Thread.sleep(CHECKPOINT_INTERVALL);
+			}
+			if(atLeastOneSnapshotComplete && !restored) {
+				throw new RuntimeException("Intended failure, to trigger restore");
+			}
+			if(restored) {
+				throw new SuccessException();
+				//throw new RuntimeException("All good");
+			}
+			someState = value; // update our state
+			return value;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			atLeastOneSnapshotComplete = true;
+		}
+	}
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * We intentionally use a user specified failure exception
+	 */
+	public static class SuccessException extends Exception {
+
+		private static final long serialVersionUID = 7073311460437532086L;
+	}
+
+	public static class NoOpSink implements SinkFunction<String> {
+		private static final long serialVersionUID = 2381410324190818620L;
+
+		@Override
+		public void invoke(String value) throws Exception {
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/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 4d10bf1..bba218f 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
@@ -43,7 +43,7 @@ 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.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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;
@@ -63,6 +63,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.UUID;
@@ -383,7 +384,7 @@ public class ChaosMonkeyITCase extends TestLogger {
 	}
 
 	public static class CheckpointedSequenceSource extends RichParallelSourceFunction<Long>
-			implements Checkpointed<Long>, CheckpointListener {
+			implements ListCheckpointed<Long>, CheckpointListener {
 
 		private static final long serialVersionUID = 0L;
 
@@ -426,18 +427,20 @@ public class ChaosMonkeyITCase extends TestLogger {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
 			LOG.info("Snapshotting state {} @ ID {}.", current, checkpointId);
-			return current;
+			return Collections.singletonList(this.current);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			LOG.info("Restoring state {}/{}", state, end);
-			current = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			LOG.info("Restoring state {}/{}", state.get(0), end);
+			this.current = state.get(0);
 		}
 
-
 		@Override
 		public void cancel() {
 			isRunning = false;
@@ -453,7 +456,7 @@ public class ChaosMonkeyITCase extends TestLogger {
 	}
 
 	public static class CountingSink extends RichSinkFunction<Long>
-			implements Checkpointed<CountingSink>, CheckpointListener {
+			implements ListCheckpointed<CountingSink>, CheckpointListener {
 
 		private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class);
 
@@ -467,7 +470,6 @@ public class ChaosMonkeyITCase extends TestLogger {
 
 		private int numberOfReceivedLastElements;
 
-
 		public CountingSink(int parallelism, long expectedFinalCount) {
 			this.expectedFinalCount = expectedFinalCount;
 			this.parallelism = parallelism;
@@ -496,16 +498,20 @@ public class ChaosMonkeyITCase extends TestLogger {
 		}
 
 		@Override
-		public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<CountingSink> snapshotState(long checkpointId, long timestamp) throws Exception {
 			LOG.info("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId);
-			return this;
+			return Collections.singletonList(this);
 		}
 
 		@Override
-		public void restoreState(CountingSink state) {
-			LOG.info("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements);
-			this.current = state.current;
-			this.numberOfReceivedLastElements = state.numberOfReceivedLastElements;
+		public void restoreState(List<CountingSink> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			CountingSink sink = state.get(0);
+			this.current = sink.current;
+			this.numberOfReceivedLastElements = sink.numberOfReceivedLastElements;
+			LOG.info("Restoring state {}:{}", sink.current, sink.numberOfReceivedLastElements);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
index 3f08b5a..f7472e1 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
@@ -42,7 +42,7 @@ 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.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 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;
@@ -63,6 +63,8 @@ import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
@@ -460,7 +462,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 	 * A checkpointed source, which emits elements from 0 to a configured number.
 	 */
 	public static class CheckpointedSequenceSource extends RichParallelSourceFunction<Long>
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 
 		private static final Logger LOG = LoggerFactory.getLogger(CheckpointedSequenceSource.class);
 
@@ -500,22 +502,26 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
 			LOG.debug("Snapshotting state {} @ ID {}.", current, checkpointId);
-			return current;
+			return Collections.singletonList(this.current);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			LOG.debug("Restoring state {}", state);
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			Long s = state.get(0);
+			LOG.debug("Restoring state {}", s);
 
 			// 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);
+			RecoveredStates.set(getRuntimeContext().getIndexOfThisSubtask(), s);
 
 			sync.countDown();
 
-			current = state;
+			current = s;
 		}
 
 		@Override
@@ -528,7 +534,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 	 * 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>,
+	public static class CountingSink implements SinkFunction<Long>, ListCheckpointed<CountingSink>,
 		CheckpointListener {
 
 		private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class);
@@ -558,16 +564,21 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		}
 
 		@Override
-		public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		public List<CountingSink> snapshotState(long checkpointId, long timestamp) throws Exception {
 			LOG.debug("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId);
-			return this;
+			return Collections.singletonList(this);
 		}
 
 		@Override
-		public void restoreState(CountingSink state) {
-			LOG.debug("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements);
-			this.current = state.current;
-			this.numberOfReceivedLastElements = state.numberOfReceivedLastElements;
+		public void restoreState(List<CountingSink> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			CountingSink s = state.get(0);
+			LOG.debug("Restoring state {}:{}", s.current, s.numberOfReceivedLastElements);
+
+			this.current = s.current;
+			this.numberOfReceivedLastElements = s.numberOfReceivedLastElements;
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/8b069fde/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 fcc3d42..987a586 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
@@ -20,6 +20,8 @@ package org.apache.flink.test.recovery;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
 
 import org.apache.commons.io.FileUtils;
@@ -29,7 +31,7 @@ import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 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.checkpoint.ListCheckpointed;
 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;
@@ -106,7 +108,7 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 	}
 
 	public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long> 
-			implements Checkpointed<Long> {
+			implements ListCheckpointed<Long> {
 
 		private static final long SLEEP_TIME = 50;
 
@@ -160,13 +162,16 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-			return collected;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.collected);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			collected = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.collected = state.get(0);
 		}
 	}
 
@@ -189,7 +194,7 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		}
 	}
 
-	private static class CheckpointedSink extends RichSinkFunction<Long> implements Checkpointed<Long> {
+	private static class CheckpointedSink extends RichSinkFunction<Long> implements ListCheckpointed<Long> {
 
 		private long stepSize;
 		private long congruence;
@@ -223,13 +228,16 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		}
 
 		@Override
-		public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return collected;
+		public List<Long> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(this.collected);
 		}
 
 		@Override
-		public void restoreState(Long state) {
-			collected = state;
+		public void restoreState(List<Long> state) throws Exception {
+			if (state.isEmpty() || state.size() > 1) {
+				throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size());
+			}
+			this.collected = state.get(0);
 		}
 	}
 }


[03/17] flink git commit: [FLINK-5150] [webui] Store metrics in sessionStorage

Posted by ch...@apache.org.
[FLINK-5150] [webui] Store metrics in sessionStorage


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

Branch: refs/heads/release-1.2
Commit: 79b68263f52690697ceec0528023d35feed0bd89
Parents: 92e3935
Author: zentol <ch...@apache.org>
Authored: Fri Jan 13 12:05:59 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:41:59 2017 +0100

----------------------------------------------------------------------
 .../web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/79b68263/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee
index 20c7439..9b9fab7 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/metrics.svc.coffee
@@ -66,13 +66,13 @@ angular.module('flinkApp')
     100
 
   @setupLS = ->
-    if !localStorage.flinkMetrics?
+    if !sessionStorage.flinkMetrics?
       @saveSetup()
 
-    @metrics = JSON.parse(localStorage.flinkMetrics)
+    @metrics = JSON.parse(sessionStorage.flinkMetrics)
 
   @saveSetup = ->
-    localStorage.flinkMetrics = JSON.stringify(@metrics)
+    sessionStorage.flinkMetrics = JSON.stringify(@metrics)
 
   @saveValue = (jobid, nodeid, value) ->
     unless @values[jobid]?


[15/17] flink git commit: [FLINK-5493] Fix String formats in FlinkDistributionOverlay and Task

Posted by ch...@apache.org.
[FLINK-5493] Fix String formats in FlinkDistributionOverlay and Task


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

Branch: refs/heads/release-1.2
Commit: 82a6b83ec4c1660dfe97fd299ac3169e5c10088b
Parents: 64c4c9d
Author: Boris Osipov <bo...@epam.com>
Authored: Thu Dec 15 12:10:26 2016 +0300
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 23:59:12 2017 +0100

----------------------------------------------------------------------
 .../clusterframework/overlays/FlinkDistributionOverlay.java    | 6 +++---
 .../main/java/org/apache/flink/runtime/taskmanager/Task.java   | 4 ++--
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/82a6b83e/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java
index a36cc67..0655554 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java
@@ -99,21 +99,21 @@ public class FlinkDistributionOverlay extends AbstractContainerOverlay {
 				flinkBinPath = new File(System.getenv(ENV_FLINK_BIN_DIR));
 			}
 			else {
-				throw new IllegalStateException(String.format("the {} environment variable must be set", ENV_FLINK_BIN_DIR));
+				throw new IllegalStateException(String.format("the %s environment variable must be set", ENV_FLINK_BIN_DIR));
 			}
 
 			if(env.containsKey(ENV_FLINK_CONF_DIR)) {
 				flinkConfPath = new File(System.getenv(ENV_FLINK_CONF_DIR));
 			}
 			else {
-				throw new IllegalStateException(String.format("the {} environment variable must be set", ENV_FLINK_CONF_DIR));
+				throw new IllegalStateException(String.format("the %s environment variable must be set", ENV_FLINK_CONF_DIR));
 			}
 
 			if(env.containsKey(ENV_FLINK_LIB_DIR)) {
 				flinkLibPath = new File(System.getenv(ENV_FLINK_LIB_DIR));
 			}
 			else {
-				throw new IllegalStateException(String.format("the {} environment variable must be set", ENV_FLINK_LIB_DIR));
+				throw new IllegalStateException(String.format("the %s environment variable must be set", ENV_FLINK_LIB_DIR));
 			}
 
 			return this;

http://git-wip-us.apache.org/repos/asf/flink/blob/82a6b83e/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index a1fb35e..e945b93 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -705,7 +705,7 @@ public class Task implements Runnable, TaskActions {
 						else {
 							if (transitionState(current, ExecutionState.FAILED, t)) {
 								// proper failure of the task. record the exception as the root cause
-								String errorMessage = String.format("Execution of {} ({}) failed.", taskNameWithSubtask, executionId);
+								String errorMessage = String.format("Execution of %s (%s) failed.", taskNameWithSubtask, executionId);
 								failureCause = t;
 								cancelInvokable();
 
@@ -1062,7 +1062,7 @@ public class Task implements Runnable, TaskActions {
 							resultPartitionId,
 							ExecutionState.RUNNING);
 					} else if (throwable instanceof PartitionProducerDisposedException) {
-						String msg = String.format("Producer {} of partition {} disposed. Cancelling execution.",
+						String msg = String.format("Producer %s of partition %s disposed. Cancelling execution.",
 							resultPartitionId.getProducerId(), resultPartitionId.getPartitionId());
 						LOG.info(msg, throwable);
 						cancelExecution();


[10/17] flink git commit: [FLINK-5417] [docs] Fix config file name in slots_parallelism.svg

Posted by ch...@apache.org.
[FLINK-5417] [docs] Fix config file name in slots_parallelism.svg


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

Branch: refs/heads/release-1.2
Commit: 7f20bd0d65cdce9e6180c20777959e4c522c55ce
Parents: fc4b45e
Author: WangTaoTheTonic <wa...@huawei.com>
Authored: Mon Jan 16 16:46:03 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:01 2017 +0100

----------------------------------------------------------------------
 docs/fig/slots_parallelism.svg | 9 ++++-----
 docs/setup/yarn_setup.md       | 2 +-
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[06/17] flink git commit: [FLINK-5329] Fix metric list being cut off

Posted by ch...@apache.org.
[FLINK-5329] Fix metric list being cut off


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

Branch: refs/heads/release-1.2
Commit: 8cdbe44aea1f7e544e4a2907e5274012b1481c57
Parents: 792f7e4
Author: zentol <ch...@apache.org>
Authored: Thu Jan 12 18:52:56 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:00 2017 +0100

----------------------------------------------------------------------
 .../app/partials/jobs/job.plan.node-list.metrics.jade          | 6 +++---
 .../web-dashboard/app/scripts/common/filters.coffee            | 6 ++++++
 flink-runtime-web/web-dashboard/app/styles/metric.styl         | 2 ++
 3 files changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8cdbe44a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.metrics.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.metrics.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.metrics.jade
index 18efbe9..fd7382f 100644
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.metrics.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.plan.node-list.metrics.jade
@@ -22,16 +22,16 @@ nav.navbar.navbar-default.navbar-secondary-additional.navbar-secondary-additiona
   .navbar-info {{ vertex.name }}
   .navbar-info {{ nodeid }}
 
-  .dropup.add-metrics(ng-if="availableMetrics.length")
+  .dropdown.add-metrics(ng-if="availableMetrics.length")
     button.btn.btn-default.navbar-btn.dropdown-toggle(type='button', data-toggle='dropdown')
       | Add metric
       | &nbsp;
       span.caret
     ul.dropdown-menu.dropdown-menu-right.metric-menu
       li(ng-repeat="metric in availableMetrics track by $index")
-        a(ng-click="addMetric(metric)") {{ metric.id }}
+        a(ng-click="addMetric(metric)") {{ metric.id | limit }}
 
-  .dropup.add-metrics(ng-if="!availableMetrics.length")
+  .dropdown.add-metrics(ng-if="!availableMetrics.length")
     button.btn.btn-default.navbar-btn.dropdown-toggle(type='button', data-toggle='dropdown', disabled='disabled')
       i No metrics available
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8cdbe44a/flink-runtime-web/web-dashboard/app/scripts/common/filters.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/common/filters.coffee b/flink-runtime-web/web-dashboard/app/scripts/common/filters.coffee
index e7e831c..67b02e3 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/common/filters.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/common/filters.coffee
@@ -54,6 +54,12 @@ angular.module('flinkApp')
     else
       if short then return days + "d " + hours + "h" else return days + "d " + hours + "h " + minutes + "m " + seconds + "s"
 
+.filter "limit", ->
+  (text) ->
+    if (text.length > 73)
+      text = text.substring(0, 35) + "..." + text.substring(text.length - 35, text.length)
+    text
+
 .filter "humanizeText", ->
   (text) ->
     # TODO: extend... a lot

http://git-wip-us.apache.org/repos/asf/flink/blob/8cdbe44a/flink-runtime-web/web-dashboard/app/styles/metric.styl
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/styles/metric.styl b/flink-runtime-web/web-dashboard/app/styles/metric.styl
index 049c1b7..972352f 100644
--- a/flink-runtime-web/web-dashboard/app/styles/metric.styl
+++ b/flink-runtime-web/web-dashboard/app/styles/metric.styl
@@ -38,7 +38,9 @@
 
       .metric-menu
         max-height: 300px
+        max-width: 900px
         overflow-y: scroll
+        text-align: right
 
 $metric-row-height = 180px + 85px
 


[04/17] flink git commit: [FLINK-5419] Make full TM metrics available through REST API

Posted by ch...@apache.org.
[FLINK-5419] Make full TM metrics available through REST API


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

Branch: refs/heads/release-1.2
Commit: f90edc483000db42caaff0de650c1f5bca11bf47
Parents: ebf4443
Author: zentol <ch...@apache.org>
Authored: Mon Jan 16 14:03:21 2017 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:00 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java | 2 +-
 .../runtime/webmonitor/metrics/TaskManagerMetricsHandler.java  | 6 +++---
 .../webmonitor/metrics/TaskManagerMetricsHandlerTest.java      | 4 ++--
 .../app/scripts/modules/taskmanager/taskmanager.svc.coffee     | 2 +-
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f90edc48/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 92c2e36..a9cb630 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
@@ -297,7 +297,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/jobs/:jobid/metrics", handler(new JobMetricsHandler(metricFetcher)))
 
 			.GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
-			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
+			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
 			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", 
 				new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
 					TaskManagerLogHandler.FileMode.LOG, config, enableSSL))

http://git-wip-us.apache.org/repos/asf/flink/blob/f90edc48/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
index a69b676..f1b2e72 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.runtime.webmonitor.metrics;
 
+import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
+
 import java.util.Map;
 
 /**
@@ -31,15 +33,13 @@ import java.util.Map;
  * {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
  */
 public class TaskManagerMetricsHandler extends AbstractMetricsHandler {
-	public static final String PARAMETER_TM_ID = "tmid";
-
 	public TaskManagerMetricsHandler(MetricFetcher fetcher) {
 		super(fetcher);
 	}
 
 	@Override
 	protected Map<String, String> getMapFor(Map<String, String> pathParams, MetricStore metrics) {
-		MetricStore.TaskManagerMetricStore taskManager = metrics.getTaskManagerMetricStore(pathParams.get(PARAMETER_TM_ID));
+		MetricStore.TaskManagerMetricStore taskManager = metrics.getTaskManagerMetricStore(pathParams.get(TaskManagersHandler.TASK_MANAGER_ID_KEY));
 		if (taskManager == null) {
 			return null;
 		} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/f90edc48/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
index a410404..12c9f3f 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java
@@ -26,7 +26,7 @@ import scala.concurrent.ExecutionContext;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.flink.runtime.webmonitor.metrics.TaskManagerMetricsHandler.PARAMETER_TM_ID;
+import static org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler.TASK_MANAGER_ID_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.powermock.api.mockito.PowerMockito.mock;
@@ -40,7 +40,7 @@ public class TaskManagerMetricsHandlerTest extends TestLogger {
 		TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(fetcher);
 
 		Map<String, String> pathParams = new HashMap<>();
-		pathParams.put(PARAMETER_TM_ID, "tmid");
+		pathParams.put(TASK_MANAGER_ID_KEY, "tmid");
 
 		Map<String, String> metrics = handler.getMapFor(pathParams, store);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f90edc48/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
index ec8bcc0..8912693 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
@@ -34,7 +34,7 @@ angular.module('flinkApp')
   @loadMetrics = (taskmanagerid) ->
     deferred = $q.defer()
 
-    $http.get(flinkConfig.jobServer + "taskmanagers/" + taskmanagerid + "/metrics")
+    $http.get(flinkConfig.jobServer + "taskmanagers/" + taskmanagerid)
     .success (data, status, headers, config) ->
       deferred.resolve(data['taskmanagers'])
 


[11/17] flink git commit: [FLINK-5321] [metrics] LocalFlinkMiniCluster starts JM MetricQS

Posted by ch...@apache.org.
[FLINK-5321] [metrics] LocalFlinkMiniCluster starts JM MetricQS


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

Branch: refs/heads/release-1.2
Commit: 2bbc92cf637b2caf612c43b0f35ba2282b5bc981
Parents: 7f20bd0
Author: zentol <ch...@apache.org>
Authored: Mon Dec 12 18:15:14 2016 +0100
Committer: zentol <ch...@apache.org>
Committed: Thu Jan 19 21:42:01 2017 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/LocalFlinkMiniCluster.scala    | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2bbc92cf/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
index 09deadc..7f3e7e6 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
@@ -130,6 +130,10 @@ class LocalFlinkMiniCluster(
       ioExecutor,
       createLeaderElectionService())
 
+    if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) {
+      metricsRegistry.get.startQueryService(system, null)
+    }
+
     val archive = system.actorOf(
       getArchiveProps(
         memoryArchivistClass,
@@ -229,7 +233,9 @@ class LocalFlinkMiniCluster(
       leaderRetrievalService,
       metricsRegistry)
 
-    metricsRegistry.startQueryService(system, resourceID)
+    if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) {
+      metricsRegistry.startQueryService(system, resourceID)
+    }
 
     system.actorOf(props, taskManagerActorName)
   }


[09/17] flink git commit: [FLINK-5417] [docs] Fix config file name in slots_parallelism.svg

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/7f20bd0d/docs/fig/slots_parallelism.svg
----------------------------------------------------------------------
diff --git a/docs/fig/slots_parallelism.svg b/docs/fig/slots_parallelism.svg
index 14e0960..11c2ec9 100644
--- a/docs/fig/slots_parallelism.svg
+++ b/docs/fig/slots_parallelism.svg
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-<svg version="1.1" viewBox="0.0 0.0 971.0 1514.0" fill="none" stroke="none" stroke-linecap="square" stroke-miterlimit="10" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink"><clipPath id="p.0"><path d="m0 0l971.0 0l0 1514.0l-971.0 0l0 -1514.0z" clip-rule="nonzero"></path>
+<svg version="1.1" viewBox="0.0 0.0 971.0 1514.0" fill="none" stroke="none" stroke-linecap="square" stroke-miterlimit="10" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd" xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"><clipPath id="p.0"><path d="m0 0l971.0 0l0 1514.0l-971.0 0l0 -1514.0z" clip-rule="nonzero"></path>
 
 </clipPath><g clip-path="url(#p.0)"><path fill="#000000" fill-opacity="0.0" d="m0 0l971.126 0l0 1514.8188l-971.126 0z" fill-rule="nonzero"></path>
 <path fill="#cfe2f3" d="m196.45932 83.89501l242.11023 0l0 264.44095l-242.11023 0z" fill-rule="nonzero"></path>
@@ -111,7 +111,6 @@ under the License.
 <path fill="#000000" d="m17.68832 158.59627l0 -13.640625l1.53125 0l0 1.28125q0.53125 -0.75 1.203125 -1.125q0.6875 -0.375 1.640625 -0.375q1.265625 0 2.234375 0.65625q0.96875 0.640625 1.453125 1.828125q0.5 1.1875 0.5 2.59375q0 1.515625 -0.546875 2.734375q-0.546875 1.203125 -1.578125 1.84375q-1.03125 0.640625 -2.171875 0.640625q-0.84375 0 -1.515625 -0.34375q-0.65625 -0.359375 -1.078125 -0.890625l0 4.796875l-1.671875 0zm1.515625 -8.65625q0 1.90625 0.765625 2.8125q0.78125 0.90625 1.875 0.90625q1.109375 0 1.890625 -0.9375q0.796875 -0.9375 0.796875 -2.921875q0 -1.875 -0.78125 -2.8125q-0.765625 -0.9375 -1.84375 -0.9375q-1.0625 0 -1.890625 1.0q-0.8125 1.0 -0.8125 2.890625zm8.844467 4.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0zm5.6
 03302 -4.921875q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm15.719467 1.3125l1.640625 0.21875q-0.265625 1.6875 -1.375 2.65625q-1.109375 0.953125 -2.734375 0.953125q-2.015625 0 -3.25 -1.3125q-1.21875 -1.328125 -1.21875 -3.796875q0 -1.59375 0.515625 -2.78125q0.53125 -1.203125 1.609375 -1.796875q1.09375 -0.609375 2.359375 -0.609375q1.609375 0 2.625 0.8125q1.015625 0.8125 1.3125 2.3125l-1.625 0.25q-0.234375 -1.0 -0.828125 -1.5q-0.59375 -0.5 -1.421875 -0.5q-1.265625 0 -2.0625 0.90625q-0.78125 0.90625 -
 0.78125 2.859375q0 1.984375 0.765625 2.890625q0.765625 0.890625 1.984375 0.890625q0.984375 0 1.640625 -0.59375q0.65625 -0.609375 0.84375 -1.859375zm9.640625 0.4375l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm8.438217 2.9375l1.6562538 -0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.562
 5q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 -0.34375 -1.078125 -0.9375q-0.3593788 -0.609375 -0.3593788 -1.328125q0 -0.65625 0.2968788 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 -0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 -0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0000038 -0.796875 -1.2812538 -2.359375zm9.328129 0l1.65625 -0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -
 1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 -0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 -0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 -0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm10.015625 -8.75l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 11.6875l0 -9.859375l1.
 671875 0l0 9.859375l-1.671875 0zm4.129196 0l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm10.078842 0.8125l1.609375 0.25q0.109375 0.75 0.578125 1.09375q0.609375 0.453125 1.6875 0.453125q1.171875 0 1.796875 -0.46875q0.625 -0.453125 0.859375 -1.28125q0.125 -0.515625 0.109375 -2.15625q-1.09375 1.296875 -2.71875 1.296875q-2.03125 0 -3.15625 -1.46875q-1.109375 -1.46875 -1.109375 -3.515625q0 -1.40625 0.515625 -2.59375q0.515625 -1.203125 1.484375 -1.84375q0.96875 -0.65625 2.265625 -0.65625q1.75 0 2.875 1.40625l0 -1.1875l1.546875 0l0 8.515625q0 2.3125 -0.46875 3.265625q-0.46875 0.96875 -1.484375 1.515625q-1.015625 0.5625 -2.5 0.5625q-1.765625 0 -2.8
 59375 -0.796875q-1.078125 -0.796875 -1.03125 -2.390625zm1.375 -5.921875q0 1.953125 0.765625 2.84375q0.78125 0.890625 1.9375 0.890625q1.140625 0 1.921875 -0.890625q0.78125 -0.890625 0.78125 -2.78125q0 -1.8125 -0.8125 -2.71875q-0.796875 -0.921875 -1.921875 -0.921875q-1.109375 0 -1.890625 0.90625q-0.78125 0.890625 -0.78125 2.671875zm13.809021 2.171875l1.65625 -0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 -0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 -0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 -0.4375q-1.15625 0 -1.64062
 5 0.390625q-0.484375 0.375 -0.484375 0.875q0 0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm9.96875 2.9375l0 -13.59375l1.671875 0l0 13.59375l-1.671875 0zm3.5510712 -4.921875q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.
 9375 -0.828125 2.828125zm12.938217 3.421875l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 -1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 -1.984375l0 -5.65625l-1.234375 0l0 -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 -0.078125zm0.85517883 -1.4375l1.65625 -0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 -0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 -0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.5156
 25l-1.625 0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 -0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm10.46875 -5.015625l0 -1.90625l1.90625 0l0 1.90625l-1.90625 0zm0 7.953125l0 -1.90625l1.90625 0l0 1.90625l-1.90625 0zm9.694733 -3.140625l1.59375 -0.15625q0.203125 1.140625 0.78125 1.65625q0.578125 0.5 1.484375 0.5q0.765625 0 1.34375 -0.34375q0.578125 -0.359375 0.953125 -0.953125q0.375 -0.59375 0.625 -1.59375q0.25 -1.0 0.25 -2.03125q0 -0.109375 -0.015625 -0.34375q-0.5 0.796875 -1.375 1.296875q-0.859375 0.5 -1.875 0.5q-1.6875 0 -2.859375 -1.21875q-1.171875 -1.234375 -1.171875 -3.234375q0 -2.07
 8125 1.21875 -3.328125q1.234375 -1.265625 3.0625 -1.265625q1.328125 0 2.421875 0.71875q1.109375 0.703125 1.671875 2.03125q0.578125 1.328125 0.578125 3.828125q0 2.609375 -0.578125 4.15625q-0.5625 1.546875 -1.6875 2.359375q-1.109375 0.796875 -2.609375 0.796875q-1.59375 0 -2.609375 -0.890625q-1.0 -0.890625 -1.203125 -2.484375zm6.828125 -6.0q0 -1.4375 -0.765625 -2.28125q-0.765625 -0.859375 -1.84375 -0.859375q-1.109375 0 -1.9375 0.921875q-0.828125 0.90625 -0.828125 2.34375q0 1.3125 0.78125 2.125q0.796875 0.796875 1.9375 0.796875q1.171875 0 1.90625 -0.796875q0.75 -0.8125 0.75 -2.25z" fill-rule="nonzero"></path>
 <path fill="#d9ead3" d="m227.5538 9.5958l99.39633 0l0 0l149.09448 0l347.88715 0l0 43.34121l0 0l0 18.574802l0 12.383202l-347.88715 0l-68.84985 17.903137l-80.24463 -17.903137l-99.39633 0l0 -12.383202l0 -18.574802l0 0z" fill-rule="nonzero"></path>
 <path stroke="#000000" stroke-width="2.0" stroke-linejoin="round" stroke-linecap="butt" d="m227.5538 9.5958l99.39633 0l0 0l149.09448 0l347.88715 0l0 43.34121l0 0l0 18.574802l0 12.383202l-347.88715 0l-68.84985 17.903137l-80.24463 -17.903137l-99.39633 0l0 -12.383202l0 -18.574802l0 0z" fill-rule="nonzero"></path>
-<path fill="#000000" d="m236.77255 21.80603l1.453125 0l0 -0.734375q0 -1.25 0.265625 -1.859375q0.265625 -0.609375 0.96875 -0.984375q0.71875 -0.390625 1.796875 -0.390625q1.109375 0 2.1875 0.328125l-0.359375 1.8125q-0.625 -0.140625 -1.203125 -0.140625q-0.5625 0 -0.8125 0.265625q-0.234375 0.265625 -0.234375 1.015625l0 0.6875l1.9375 0l0 2.0625l-1.9375 0l0 7.796875l-2.609375 0l0 -7.796875l-1.453125 0l0 -2.0625zm7.353302 9.859375l0 -13.59375l2.609375 0l0 13.59375l-2.609375 0zm5.183304 -11.1875l0 -2.40625l2.609375 0l0 2.40625l-2.609375 0zm0 11.1875l0 -9.859375l2.609375 0l0 9.859375l-2.609375 0zm14.152054 0l-2.609375 0l0 -5.03125q0 -1.59375 -0.171875 -2.0625q-0.15625 -0.46875 -0.53125 -0.71875q-0.375 -0.265625 -0.90625 -0.265625q-0.6875 0 -1.234375 0.375q-0.53125 0.359375 -0.734375 0.984375q-0.1875 0.609375 -0.1875 2.25l0 4.46875l-2.609375 0l0 -9.859375l2.421875 0l0 1.453125q1.296875 -1.671875 3.25 -1.671875q0.859375 0 1.578125 0.3125q0.71875 0.3125 1.078125 0.796875q0.359375 0.484375 0.5 1.
 09375q0.15625 0.609375 0.15625 1.75l0 6.125zm2.333496 0l0 -13.59375l2.609375 0l0 7.203125l3.0625 -3.46875l3.203125 0l-3.375 3.609375l3.609375 6.25l-2.796875 0l-2.484375 -4.421875l-1.21875 1.265625l0 3.15625l-2.609375 0zm10.188202 -3.625l0 -2.609375l5.109375 0l0 2.609375l-5.109375 0zm15.087677 -3.3125l-2.5625 0.46875q-0.140625 -0.78125 -0.59375 -1.171875q-0.453125 -0.390625 -1.1875 -0.390625q-0.984375 0 -1.5625 0.671875q-0.578125 0.671875 -0.578125 2.25q0 1.765625 0.578125 2.484375q0.59375 0.71875 1.59375 0.71875q0.75 0 1.21875 -0.421875q0.46875 -0.421875 0.671875 -1.453125l2.546875 0.4375q-0.390625 1.765625 -1.53125 2.671875q-1.125 0.890625 -3.03125 0.890625q-2.15625 0 -3.453125 -1.359375q-1.28125 -1.359375 -1.28125 -3.78125q0 -2.4375 1.296875 -3.796875q1.296875 -1.359375 3.484375 -1.359375q1.796875 0 2.859375 0.78125q1.0625 0.765625 1.53125 2.359375zm1.1882324 1.875q0 -1.296875 0.640625 -2.515625q0.640625 -1.21875 1.8125 -1.859375q1.171875 -0.640625 2.609375 -0.640625q2.25 0 3.6718
 75 1.453125q1.421875 1.453125 1.421875 3.671875q0 2.234375 -1.4375 3.703125q-1.4375 1.46875 -3.625 1.46875q-1.359375 0 -2.59375 -0.609375q-1.21875 -0.609375 -1.859375 -1.796875q-0.640625 -1.1875 -0.640625 -2.875zm2.671875 0.140625q0 1.46875 0.6875 2.25q0.703125 0.765625 1.71875 0.765625q1.015625 0 1.703125 -0.765625q0.703125 -0.78125 0.703125 -2.265625q0 -1.453125 -0.703125 -2.234375q-0.6875 -0.78125 -1.703125 -0.78125q-1.015625 0 -1.71875 0.78125q-0.6875 0.78125 -0.6875 2.25zm18.28659 4.921875l-2.609375 0l0 -5.03125q0 -1.59375 -0.171875 -2.0625q-0.15625 -0.46875 -0.53125 -0.71875q-0.375 -0.265625 -0.90625 -0.265625q-0.6875 0 -1.234375 0.375q-0.53125 0.359375 -0.734375 0.984375q-0.1875 0.609375 -0.1875 2.25l0 4.46875l-2.609375 0l0 -9.859375l2.421875 0l0 1.453125q1.296875 -1.671875 3.25 -1.671875q0.859375 0 1.578125 0.3125q0.71875 0.3125 1.078125 0.796875q0.359375 0.484375 0.5 1.09375q0.15625 0.609375 0.15625 1.75l0 6.125zm1.2866211 -9.859375l1.453125 0l0 -0.734375q0 -1.25 0.265625 -
 1.859375q0.265625 -0.609375 0.96875 -0.984375q0.71875 -0.390625 1.796875 -0.390625q1.109375 0 2.1875 0.328125l-0.359375 1.8125q-0.625 -0.140625 -1.203125 -0.140625q-0.5625 0 -0.8125 0.265625q-0.234375 0.265625 -0.234375 1.015625l0 0.6875l1.9375 0l0 2.0625l-1.9375 0l0 7.796875l-2.609375 0l0 -7.796875l-1.453125 0l0 -2.0625zm7.353302 -1.328125l0 -2.40625l2.609375 0l0 2.40625l-2.609375 0zm0 11.1875l0 -9.859375l2.609375 0l0 9.859375l-2.609375 0zm4.9489136 0.65625l2.96875 0.359375q0.078125 0.515625 0.34375 0.703125q0.375 0.28125 1.171875 0.28125q1.03125 0 1.53125 -0.296875q0.34375 -0.203125 0.515625 -0.65625q0.125 -0.328125 0.125 -1.203125l0 -1.4375q-1.171875 1.59375 -2.953125 1.59375q-1.984375 0 -3.140625 -1.671875q-0.90625 -1.328125 -0.90625 -3.3125q0 -2.46875 1.1875 -3.78125q1.1875 -1.3125 2.96875 -1.3125q1.828125 0 3.015625 1.609375l0 -1.390625l2.4375 0l0 8.84375q0 1.75 -0.296875 2.609375q-0.28125 0.859375 -0.796875 1.34375q-0.515625 0.5 -1.390625 0.78125q-0.859375 0.28125 -2.1875 0.2
 8125q-2.515625 0 -3.5625 -0.859375q-1.046875 -0.859375 -1.046875 -2.171875q0 -0.140625 0.015625 -0.3125zm2.328125 -5.78125q0 1.5625 0.609375 2.296875q0.609375 0.71875 1.5 0.71875q0.953125 0 1.609375 -0.734375q0.65625 -0.75 0.65625 -2.21875q0 -1.53125 -0.640625 -2.265625q-0.625 -0.75 -1.578125 -0.75q-0.9375 0 -1.546875 0.734375q-0.609375 0.71875 -0.609375 2.21875zm9.302246 5.125l0 -2.609375l2.609375 0l0 2.609375l-2.609375 0zm3.9489136 -9.859375l2.78125 0l2.359375 7.0l2.296875 -7.0l2.703125 0l-3.484375 9.484375l-0.625 1.71875q-0.34375 0.859375 -0.65625 1.3125q-0.296875 0.46875 -0.703125 0.75q-0.40625 0.28125 -1.0 0.4375q-0.59375 0.15625 -1.328125 0.15625q-0.75 0 -1.46875 -0.15625l-0.234375 -2.046875q0.609375 0.125 1.09375 0.125q0.921875 0 1.34375 -0.53125q0.4375 -0.53125 0.671875 -1.359375l-3.75 -9.890625zm13.563232 3.015625l-2.359375 -0.4375q0.390625 -1.421875 1.359375 -2.109375q0.984375 -0.6875 2.90625 -0.6875q1.734375 0 2.59375 0.421875q0.859375 0.40625 1.203125 1.046875q0.34375 0.
 625 0.34375 2.328125l-0.03125 3.046875q0 1.296875 0.125 1.921875q0.125 0.609375 0.46875 1.3125l-2.578125 0q-0.09375 -0.265625 -0.25 -0.765625q-0.0625 -0.234375 -0.09375 -0.3125q-0.65625 0.65625 -1.421875 0.984375q-0.765625 0.3125 -1.625 0.3125q-1.515625 0 -2.40625 -0.8125q-0.875 -0.828125 -0.875 -2.09375q0 -0.84375 0.390625 -1.484375q0.40625 -0.65625 1.125 -1.0q0.71875 -0.359375 2.078125 -0.625q1.828125 -0.328125 2.53125 -0.625l0 -0.265625q0 -0.75 -0.375 -1.0625q-0.359375 -0.328125 -1.390625 -0.328125q-0.703125 0 -1.09375 0.28125q-0.390625 0.265625 -0.625 0.953125zm3.484375 2.109375q-0.5 0.171875 -1.59375 0.40625q-1.078125 0.234375 -1.40625 0.453125q-0.515625 0.359375 -0.515625 0.921875q0 0.546875 0.40625 0.953125q0.40625 0.390625 1.046875 0.390625q0.703125 0 1.34375 -0.46875q0.46875 -0.359375 0.625 -0.859375q0.09375 -0.34375 0.09375 -1.28125l0 -0.515625zm4.750702 -5.125l2.40625 0l0 1.359375q1.28125 -1.578125 3.0625 -1.578125q0.953125 0 1.640625 0.390625q0.703125 0.390625 1.140625 1
 .1875q0.65625 -0.796875 1.40625 -1.1875q0.75 -0.390625 1.609375 -0.390625q1.078125 0 1.828125 0.453125q0.75 0.4375 1.125 1.28125q0.265625 0.640625 0.265625 2.046875l0 6.296875l-2.609375 0l0 -5.625q0 -1.46875 -0.265625 -1.90625q-0.359375 -0.546875 -1.109375 -0.546875q-0.546875 0 -1.03125 0.34375q-0.484375 0.328125 -0.703125 0.96875q-0.203125 0.640625 -0.203125 2.03125l0 4.734375l-2.609375 0l0 -5.40625q0 -1.4375 -0.140625 -1.84375q-0.140625 -0.421875 -0.4375 -0.625q-0.28125 -0.203125 -0.78125 -0.203125q-0.609375 0 -1.09375 0.328125q-0.484375 0.3125 -0.6875 0.9375q-0.203125 0.609375 -0.203125 2.03125l0 4.78125l-2.609375 0l0 -9.859375zm16.77591 9.859375l0 -13.59375l2.609375 0l0 13.59375l-2.609375 0zm5.6832886 -7.25l0 -2.609375l2.609375 0l0 2.609375l-2.609375 0zm0 7.25l0 -2.609375l2.609375 0l0 2.609375l-2.609375 0z" fill-rule="nonzero"></path>
 <path fill="#000000" d="m240.64755 45.634155l4.234375 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-4.234375 0l0 5.1875q0 0.6875 0.546875 1.140625q0.546875 0.453125 1.59375 0.453125q0.78125 0 1.703125 -0.234375q0.921875 -0.25 1.4375 -0.546875q0.1875 -0.109375 0.296875 -0.109375q0.15625 0 0.265625 0.125q0.109375 0.109375 0.109375 0.265625q0 0.140625 -0.125 0.25q-0.296875 0.3125 -1.4375 0.671875q-1.140625 0.359375 -2.1875 0.359375q-1.375 0 -2.1875 -0.640625q-0.796875 -0.640625 -0.796875 -1.734375l0 -5.1875l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0l0 -2.296875q0 -0.28125 0.109375 -0.40625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.125 0.109375 0.40625l0 2.296875zm15.245605 8.03125l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.7
 5q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.03125 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.484375 -1.78125 -0.484375q-0.90625 0 -2.546875 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4375 -0.390625 2.1875 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.703125 0 -2.65625 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.5625 0.484375 1.625 0.484375q1.0 0 1.859375 -0.40625q0.875 -0.40625 1.796875 -1.28125l0 -1.71875zm11.745605 -3.65625q0 -0.265625 0.109375 -0.375q0.109375 -0.125 0.28125 
 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.40625l0 1.328125q0 0.28125 -0.109375 0.40625q-0.109375 0.109375 -0.28125 0.109375q-0.171875 0 -0.28125 -0.09375q-0.09375 -0.109375 -0.109375 -0.34375q-0.0625 -0.546875 -0.578125 -0.90625q-0.765625 -0.53125 -2.015625 -0.53125q-1.3125 0 -2.03125 0.53125q-0.546875 0.40625 -0.546875 0.890625q0 0.5625 0.640625 0.9375q0.453125 0.25 1.703125 0.390625q1.609375 0.171875 2.25 0.40625q0.890625 0.328125 1.328125 0.90625q0.453125 0.5625 0.453125 1.234375q0 0.984375 -0.953125 1.765625q-0.953125 0.78125 -2.8125 0.78125q-1.84375 0 -3.015625 -0.9375q0 0.3125 -0.046875 0.40625q-0.03125 0.09375 -0.125 0.15625q-0.09375 0.0625 -0.21875 0.0625q-0.171875 0 -0.28125 -0.109375q-0.109375 -0.125 -0.109375 -0.40625l0 -1.609375q0 -0.28125 0.109375 -0.390625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.296875q0 0.421875 0.21875 0.71875q0.328125 0.4375 1.03125 0.734375q0.71875 0.296875 1.75 0.296875q1.515625 0 2.2
 5 -0.5625q0.75 -0.578125 0.75 -1.203125q0 -0.71875 -0.75 -1.15625q-0.765625 -0.4375 -2.21875 -0.578125q-1.453125 -0.15625 -2.078125 -0.390625q-0.625 -0.25 -0.984375 -0.734375q-0.359375 -0.484375 -0.359375 -1.03125q0 -1.015625 0.984375 -1.59375q0.984375 -0.59375 2.359375 -0.59375q1.609375 0 2.625 0.78125zm7.2456055 4.0625l0 3.46875l-1.828125 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.046875 0l0 -10.078125l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0l0 7.171875l3.3125 -2.78125l-0.40625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.5 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-0.90625 0l-3.046875 2.546875l3.859375 3.921875l0.890625 0q0.296
 875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-2.515625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l0.546875 0l-3.34375 -3.4375l-0.890625 0.75zm9.433105 -4.5625l0 0.78125q0.984375 -1.0625 1.984375 -1.0625q0.609375 0 1.0625 0.328125q0.453125 0.3125 0.765625 0.96875q0.515625 -0.65625 1.046875 -0.96875q0.53125 -0.328125 1.078125 -0.328125q0.84375 0 1.34375 0.546875q0.65625 0.703125 0.65625 1.546875l0 5.4375l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-1.4375 0l0 -6.15625q0 -0.59375 -0.359375 -0.984375q-0.359375 -0.390625 -0.828125 -0.390625q-0.4375 0 -0.90625 0.328125q-0.46875 0.3125 -1.078125 1.25l0 5.171875l0.65625 0q0.265625 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.390625 0.109375l-1.4375
  0l0 -6.09375q0 -0.625 -0.375 -1.03125q-0.359375 -0.40625 -0.8125 -0.40625q-0.421875 0 -0.828125 0.28125q-0.5625 0.375 -1.1875 1.296875l0 5.171875l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-2.09375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l0.65625 0l0 -6.46875l-0.65625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0zm17.183105 8.03125l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.75q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.03125 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.484375 -1.78125 -0.484375q-0.90625 0 -2.546875 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0
 .109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4375 -0.390625 2.1875 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.703125 0 -2.65625 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.5625 0.484375 1.625 0.484375q1.0 0 1.859375 -0.40625q0.875 -0.40625 1.796875 -1.28125l0 -1.71875zm6.6518555 -4.15625l0 1.171875q0.8125 -0.828125 1.46875 -1.140625q0.65625 -0.3125 1.484375 -0.3125q0.890625 0 1.625 0.375q0.515625 0.28125 0.9375 0.9375q0.421875 0.640625 0.421875 1.3125l0 4.90625l0.65625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-2.078125 0q-0.296875 0 -0.421875 -0.109375q-0.1093
 75 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l0.640625 0l0 -4.78125q0 -0.828125 -0.609375 -1.390625q-0.59375 -0.578125 -1.609375 -0.578125q-0.765625 0 -1.328125 0.3125q-0.5625 0.3125 -1.609375 1.546875l0 4.890625l0.875 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-2.546875 0q-0.265625 0 -0.390625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.390625 -0.109375l0.890625 0l0 -6.46875l-0.65625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0zm16.151855 8.03125l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.75q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.03125 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.4
 84375 -1.78125 -0.484375q-0.90625 0 -2.546875 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4375 -0.390625 2.1875 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.703125 0 -2.65625 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.5625 0.484375 1.625 0.484375q1.0 0 1.859375 -0.40625q0.875 -0.40625 1.796875 -1.28125l0 -1.71875zm11.79248 -2.734375l0 -1.421875l1.828125 0q0.265625 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.390625 0.109375l-1.0625 0l0 7.78125q0 0.78125 -0.328125 1.40625q-0.21875 0.40625 -0.75 0.828125q-0.515625 0.
 4375 -0.9375 0.625q-0.421875 0.1875 -1.140625 0.1875l-2.203125 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.09375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.234375 0q0.6875 0 1.234375 -0.359375q0.546875 -0.34375 0.921875 -1.015625q0.203125 -0.390625 0.203125 -1.0l0 -2.328125q-1.265625 1.71875 -3.234375 1.71875q-1.609375 0 -2.78125 -1.1875q-1.15625 -1.203125 -1.15625 -2.890625q0 -1.671875 1.15625 -2.859375q1.171875 -1.1875 2.78125 -1.1875q1.96875 0 3.234375 1.703125zm0 2.34375q0 -1.359375 -0.953125 -2.3125q-0.9375 -0.953125 -2.25 -0.953125q-1.296875 0 -2.234375 0.953125q-0.9375 0.953125 -0.9375 2.3125q0 1.375 0.9375 2.328125q0.9375 0.953125 2.234375 0.953125q1.3125 0 2.25 -0.953125q0.953125 -0.953125 0.953125 -2.328125zm12.995605 0.4375l-7.953125 0q0.21875 1.515625 1.28125 2.4375q1.0625 0.921875 2.625 0.921875q0.875 0 1.828125 -0.28125q0.953125 -0.296875 1.5625 -0.765625q0.171875 -0.140625 0.296875 -0.140625q0.15625 0 0.265625 0.125q0.109375 0.109375 
 0.109375 0.265625q0 0.15625 -0.15625 0.296875q-0.4375 0.46875 -1.578125 0.875q-1.140625 0.40625 -2.328125 0.40625q-2.0 0 -3.34375 -1.3125q-1.34375 -1.3125 -1.34375 -3.1875q0 -1.6875 1.25 -2.90625q1.265625 -1.21875 3.125 -1.21875q1.90625 0 3.140625 1.25q1.234375 1.25 1.21875 3.234375zm-0.796875 -0.796875q-0.234375 -1.28125 -1.21875 -2.09375q-0.984375 -0.8125 -2.34375 -0.8125q-1.375 0 -2.359375 0.8125q-0.96875 0.796875 -1.21875 2.09375l7.140625 0zm7.0424805 -3.40625l0 1.96875q1.53125 -1.375 2.28125 -1.765625q0.75 -0.40625 1.390625 -0.40625q0.703125 0 1.296875 0.46875q0.59375 0.46875 0.59375 0.71875q0 0.171875 -0.109375 0.296875q-0.109375 0.109375 -0.296875 0.109375q-0.09375 0 -0.15625 -0.03125q-0.0625 -0.03125 -0.234375 -0.21875q-0.328125 -0.3125 -0.578125 -0.4375q-0.234375 -0.125 -0.46875 -0.125q-0.5 0 -1.21875 0.40625q-0.71875 0.40625 -2.5 2.0l0 4.265625l3.453125 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.
 109375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.265625q0 -0.15625 0.109375 -0.265625q0.109375 -0.109375 0.40625 -0.109375l1.875 0l0 -6.5l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.21875 0zm12.16748 5.796875l0.34375 0q0.515625 0 0.890625 0.375q0.375 0.359375 0.375 0.875q0 0.53125 -0.375 0.90625q-0.375 0.359375 -0.890625 0.359375l-0.34375 0q-0.515625 0 -0.890625 -0.359375q-0.359375 -0.375 -0.359375 -0.890625q0 -0.546875 0.375 -0.90625q0.375 -0.359375 0.875 -0.359375zm9.058105 -5.796875l0 1.171875q0.8125 -0.828125 1.46875 -1.140625q0.65625 -0.3125 1.484375 -0.3125q0.890625 0 1.625 0.375q0.515625 0.28125 0.9375 0.9375q0.421875 0.640625 0.421875 1.3125l0 4.90625l0.65625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-2.078125 0q-0.296875 0 -0.421875 -0.109375q-0.109375 -0.109375 -0.109375 -0.28
 125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l0.640625 0l0 -4.78125q0 -0.828125 -0.609375 -1.390625q-0.59375 -0.578125 -1.609375 -0.578125q-0.765625 0 -1.328125 0.3125q-0.5625 0.3125 -1.609375 1.546875l0 4.890625l0.875 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-2.546875 0q-0.265625 0 -0.390625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.390625 -0.109375l0.890625 0l0 -6.46875l-0.65625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0zm16.589355 8.03125l0 -1.140625q-1.609375 1.453125 -3.453125 1.453125q-1.140625 0 -1.734375 -0.625q-0.78125 -0.8125 -0.78125 -1.90625l0 -5.03125l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0l0 5.8125q0 0.765625 0.48437
 5 1.265625q0.484375 0.484375 1.203125 0.484375q1.90625 0 3.5 -1.75l0 -5.03125l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.203125 0l0 7.25l0.671875 0q0.265625 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.390625 0.109375l-1.4375 0zm5.1831055 -8.03125l0 0.78125q0.984375 -1.0625 1.984375 -1.0625q0.609375 0 1.0625 0.328125q0.453125 0.3125 0.765625 0.96875q0.515625 -0.65625 1.046875 -0.96875q0.53125 -0.328125 1.078125 -0.328125q0.84375 0 1.34375 0.546875q0.65625 0.703125 0.65625 1.546875l0 5.4375l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-1.4375 0l0 -6.15625q0 -0.59375 -0.359375 -0.984375q-0.359375 -0.390625 -0.828125 -0.390625q-0.4375 0 -0.90625 0.328125q-0.46875 0.3125 -1.078125 1.25l0 5.171875l0.65625 0q0.265625 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125
 q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.390625 0.109375l-1.4375 0l0 -6.09375q0 -0.625 -0.375 -1.03125q-0.359375 -0.40625 -0.8125 -0.40625q-0.421875 0 -0.828125 0.28125q-0.5625 0.375 -1.1875 1.296875l0 5.171875l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-2.09375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l0.65625 0l0 -6.46875l-0.65625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0zm12.01123 -3.609375l0 5.171875q1.40625 -1.84375 3.40625 -1.84375q1.703125 0 2.921875 1.25q1.21875 1.234375 1.21875 3.03125q0 1.8125 -1.234375 3.078125q-1.234375 1.265625 -2.90625 1.265625q-2.046875 0 -3.40625 -1.84375l0 1.53125l-1.828125 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.10
 9375 0.40625 -0.109375l1.046875 0l0 -10.078125l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0zm6.765625 7.640625q0 -1.46875 -1.015625 -2.5q-1.015625 -1.03125 -2.359375 -1.03125q-1.359375 0 -2.375 1.03125q-1.0 1.03125 -1.0 2.5q0 1.46875 1.0 2.5q1.015625 1.03125 2.375 1.03125q1.34375 0 2.359375 -1.03125q1.015625 -1.03125 1.015625 -2.5zm11.79248 0.171875l-7.953125 0q0.21875 1.515625 1.28125 2.4375q1.0625 0.921875 2.625 0.921875q0.875 0 1.828125 -0.28125q0.953125 -0.296875 1.5625 -0.765625q0.171875 -0.140625 0.296875 -0.140625q0.15625 0 0.265625 0.125q0.109375 0.109375 0.109375 0.265625q0 0.15625 -0.15625 0.296875q-0.4375 0.46875 -1.578125 0.875q-1.140625 0.40625 -2.328125 0.40625q-2.0 0 -3.34375 -1.3125q-1.34375 -1.3125 -1.34375 -3.1875q0 -1.6875 1.25 -2.90625q1.265625 -1.21875 3.125 -1.21875q1.90625 0 3.140625 1.25q1.234375 1.25 1.21875 3.234375zm-0.796875 -0.796875q-0.234375 -
 1.28125 -1.21875 -2.09375q-0.984375 -0.8125 -2.34375 -0.8125q-1.375 0 -2.359375 0.8125q-0.96875 0.796875 -1.21875 2.09375l7.140625 0zm7.0424805 -3.40625l0 1.96875q1.53125 -1.375 2.28125 -1.765625q0.75 -0.40625 1.390625 -0.40625q0.703125 0 1.296875 0.46875q0.59375 0.46875 0.59375 0.71875q0 0.171875 -0.109375 0.296875q-0.109375 0.109375 -0.296875 0.109375q-0.09375 0 -0.15625 -0.03125q-0.0625 -0.03125 -0.234375 -0.21875q-0.328125 -0.3125 -0.578125 -0.4375q-0.234375 -0.125 -0.46875 -0.125q-0.5 0 -1.21875 0.40625q-0.71875 0.40625 -2.5 2.0l0 4.265625l3.453125 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.109375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.265625q0 -0.15625 0.109375 -0.265625q0.109375 -0.109375 0.40625 -0.109375l1.875 0l0 -6.5l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.21875 0zm1
 7.10498 2.625q0 1.5625 -0.640625 2.90625q-0.625 1.328125 -1.734375 2.078125q-1.109375 0.734375 -2.375 0.734375q-1.890625 0 -3.328125 -1.59375q-1.421875 -1.609375 -1.421875 -4.125q0 -2.5 1.421875 -4.09375q1.4375 -1.609375 3.328125 -1.609375q1.265625 0 2.359375 0.75q1.109375 0.734375 1.75 2.0625q0.640625 1.328125 0.640625 2.890625zm-0.796875 0q0 -1.890625 -1.109375 -3.40625q-1.09375 -1.515625 -2.84375 -1.515625q-1.640625 0 -2.8125 1.453125q-1.15625 1.4375 -1.15625 3.46875q0 2.171875 1.21875 3.5625q1.21875 1.375 2.75 1.375q1.75 0 2.84375 -1.5q1.109375 -1.515625 1.109375 -3.4375zm6.9174805 -1.84375l0 6.46875l3.421875 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-6.09375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.875 0l0 -6.46875l-1.6875 0q-0.265625 0 -0.390625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.1
 25 -0.28125q0.125 -0.109375 0.390625 -0.109375l1.6875 0l0 -1.1875q0 -0.984375 0.796875 -1.703125q0.8125 -0.71875 2.125 -0.71875q1.109375 0 2.359375 0.203125q0.46875 0.078125 0.5625 0.1875q0.09375 0.09375 0.09375 0.25q0 0.171875 -0.109375 0.28125q-0.109375 0.109375 -0.296875 0.109375q-0.0625 0 -0.25 -0.03125q-1.390625 -0.21875 -2.359375 -0.21875q-1.015625 0 -1.578125 0.5q-0.546875 0.5 -0.546875 1.140625l0 1.1875l3.640625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-3.640625 0zm12.308105 -2.828125l0 9.296875l2.015625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-4.828125 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.015625 0l0 -9.296875l-3.1875 0l0 2.703125q0 0.28125 -0.109375 0.40625q-0.109375 0.125 -0.28125 0.125q-0.171875 0 -0.28125 -0.125q-0.109375 -0
 .125 -0.109375 -0.40625l0 -3.484375l8.734375 0l0 3.484375q0 0.28125 -0.109375 0.40625q-0.109375 0.125 -0.28125 0.125q-0.1875 0 -0.296875 -0.125q-0.09375 -0.125 -0.09375 -0.40625l0 -2.703125l-3.1875 0zm13.245605 10.078125l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.75q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.03125 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.484375 -1.78125 -0.484375q-0.90625 0 -2.546875 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4375 -0.390625 2.1875 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.
 15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.703125 0 -2.65625 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.5625 0.484375 1.625 0.484375q1.0 0 1.859375 -0.40625q0.875 -0.40625 1.796875 -1.28125l0 -1.71875zm11.745605 -3.65625q0 -0.265625 0.109375 -0.375q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.40625l0 1.328125q0 0.28125 -0.109375 0.40625q-0.109375 0.109375 -0.28125 0.109375q-0.171875 0 -0.28125 -0.09375q-0.09375 -0.109375 -0.109375 -0.34375q-0.0625 -0.546875 -0.578125 -0.90625q-0.765625 -0.53125 -2.015625 -0.53125q-1.3125 0 -2.03125 0.53125q-0.546875 0.40625 -0.546875 0.890625q0 0.5625 0.640625 0.9375q0.453125 0.25 1.703125 0.390625q1.609375 0.171875 2.25 0.40625q0.890625 0.328125 1.328125 0.90625q0.453125 0.5625 0.453125 1.234375q0 0.984375 -0.953125 1.765625q-0.953125 0.78125 -2.8125 0.78125q-1.84375 0 -3.015625 -0.9375q0 0.3125 -0.046875 0.40625q-0.03125 0.09375 -0.125 0.15625q-0.09375 0.0625 -0.
 21875 0.0625q-0.171875 0 -0.28125 -0.109375q-0.109375 -0.125 -0.109375 -0.40625l0 -1.609375q0 -0.28125 0.109375 -0.390625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.296875q0 0.421875 0.21875 0.71875q0.328125 0.4375 1.03125 0.734375q0.71875 0.296875 1.75 0.296875q1.515625 0 2.25 -0.5625q0.75 -0.578125 0.75 -1.203125q0 -0.71875 -0.75 -1.15625q-0.765625 -0.4375 -2.21875 -0.578125q-1.453125 -0.15625 -2.078125 -0.390625q-0.625 -0.25 -0.984375 -0.734375q-0.359375 -0.484375 -0.359375 -1.03125q0 -1.015625 0.984375 -1.59375q0.984375 -0.59375 2.359375 -0.59375q1.609375 0 2.625 0.78125zm7.2456055 4.0625l0 3.46875l-1.828125 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.046875 0l0 -10.078125l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0l0 7.171875l3.3125 -2.78125l-0.40625 0q
 -0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.5 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-0.90625 0l-3.046875 2.546875l3.859375 3.921875l0.890625 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-2.515625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l0.546875 0l-3.34375 -3.4375l-0.890625 0.75zm15.761261 -6.609375l0 -0.25q0 -0.28125 0.109375 -0.40625q0.109375 -0.125 0.28125 -0.125q0.1875 0 0.28125 0.125q0.109375 0.125 0.109375 0.40625l0 1.984375q0 0.265625 -0.109375 0.390625q-0.09375 0.125 -0.28125 0.125q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.125 -0.359375q-0.046875 -0.78125 -0.796875 -1.40625q-0.75 -0.625 -1.984375 -0.625q-1.28125 0 -2.046875 0.6
 71875q-0.765625 0.65625 -0.765625 1.5625q0 0.453125 0.21875 0.84375q0.21875 0.375 0.5625 0.609375q0.359375 0.234375 0.796875 0.390625q0.453125 0.140625 1.390625 0.296875q1.59375 0.25 2.1875 0.53125q0.8125 0.359375 1.21875 1.0q0.421875 0.640625 0.421875 1.515625q0 1.328125 -1.078125 2.28125q-1.0625 0.9375 -2.859375 0.9375q-2.03125 0 -3.25 -1.28125l0 0.453125q0 0.28125 -0.109375 0.40625q-0.109375 0.109375 -0.28125 0.109375q-0.171875 0 -0.28125 -0.109375q-0.10940552 -0.125 -0.10940552 -0.40625l0 -2.15625q0 -0.28125 0.10940552 -0.40625q0.109375 -0.125 0.28125 -0.125q0.15625 0 0.265625 0.109375q0.109375 0.109375 0.125 0.359375q0.046875 0.859375 0.90625 1.5625q0.875 0.703125 2.34375 0.703125q1.453125 0 2.296875 -0.734375q0.859375 -0.734375 0.859375 -1.734375q0 -0.609375 -0.328125 -1.09375q-0.328125 -0.484375 -0.96875 -0.75q-0.4375 -0.203125 -1.84375 -0.4375q-1.9375 -0.328125 -2.796875 -1.0q-0.859375 -0.671875 -0.859375 -1.90625q0 -1.21875 0.984375 -2.09375q0.984375 -0.890625 2.578125 -0.8
 90625q1.640625 0 2.8125 1.03125zm9.01123 -1.5625l0 10.859375l3.0625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.90625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l3.0625 0l0 -10.078125l-2.234375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l3.015625 0zm15.370605 7.640625q0 1.78125 -1.28125 3.046875q-1.28125 1.265625 -3.078125 1.265625q-1.8125 0 -3.09375 -1.265625q-1.265625 -1.28125 -1.265625 -3.046875q0 -1.78125 1.265625 -3.046875q1.28125 -1.265625 3.09375 -1.265625q1.796875 0 3.078125 1.265625q1.28125 1.25 1.28125 3.046875zm-0.796875 0q0 -1.46875 -1.046875 -2.5q-1.03125 -1.03125 -2.515625 -1.03125q-1.484375 0 -2.53125 1.046875q-1.046875 1.03125 -1.046875 2.484375q0 1.453125 1.046875 2.5q1.046875 1.03125 2.53125 1.03125q1.484375 0 2.515
 625 -1.03125q1.046875 -1.046875 1.046875 -2.5zm6.2299805 -4.03125l4.234375 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-4.234375 0l0 5.1875q0 0.6875 0.546875 1.140625q0.546875 0.453125 1.59375 0.453125q0.78125 0 1.703125 -0.234375q0.921875 -0.25 1.4375 -0.546875q0.1875 -0.109375 0.296875 -0.109375q0.15625 0 0.265625 0.125q0.109375 0.109375 0.109375 0.265625q0 0.140625 -0.125 0.25q-0.296875 0.3125 -1.4375 0.671875q-1.140625 0.359375 -2.1875 0.359375q-1.375 0 -2.1875 -0.640625q-0.796875 -0.640625 -0.796875 -1.734375l0 -5.1875l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0l0 -2.296875q0 -0.28125 0.109375 -0.40625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.125 0.109375 0.40625l0 2.296875zm15.589355 0.5q0 -0.265625 0.109375 -0.375q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.1
 25q0.109375 0.109375 0.109375 0.40625l0 1.328125q0 0.28125 -0.109375 0.40625q-0.109375 0.109375 -0.28125 0.109375q-0.171875 0 -0.28125 -0.09375q-0.09375 -0.109375 -0.109375 -0.34375q-0.0625 -0.546875 -0.578125 -0.90625q-0.765625 -0.53125 -2.015625 -0.53125q-1.3125 0 -2.03125 0.53125q-0.546875 0.40625 -0.546875 0.890625q0 0.5625 0.640625 0.9375q0.453125 0.25 1.703125 0.390625q1.609375 0.171875 2.25 0.40625q0.890625 0.328125 1.328125 0.90625q0.453125 0.5625 0.453125 1.234375q0 0.984375 -0.953125 1.765625q-0.953125 0.78125 -2.8125 0.78125q-1.84375 0 -3.015625 -0.9375q0 0.3125 -0.046875 0.40625q-0.03125 0.09375 -0.125 0.15625q-0.09375 0.0625 -0.21875 0.0625q-0.171875 0 -0.28125 -0.109375q-0.109375 -0.125 -0.109375 -0.40625l0 -1.609375q0 -0.28125 0.109375 -0.390625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.296875q0 0.421875 0.21875 0.71875q0.328125 0.4375 1.03125 0.734375q0.71875 0.296875 1.75 0.296875q1.515625 0 2.25 -0.5625q0.75 -0.578125 0.75
  -1.203125q0 -0.71875 -0.75 -1.15625q-0.765625 -0.4375 -2.21875 -0.578125q-1.453125 -0.15625 -2.078125 -0.390625q-0.625 -0.25 -0.984375 -0.734375q-0.359375 -0.484375 -0.359375 -1.03125q0 -1.015625 0.984375 -1.59375q0.984375 -0.59375 2.359375 -0.59375q1.609375 0 2.625 0.78125zm8.651855 -0.5l0.34375 0q0.515625 0 0.890625 0.375q0.375 0.359375 0.375 0.875q0 0.546875 -0.375 0.90625q-0.375 0.359375 -0.890625 0.359375l-0.34375 0q-0.515625 0 -0.890625 -0.359375q-0.359375 -0.375 -0.359375 -0.890625q0 -0.546875 0.375 -0.90625q0.375 -0.359375 0.875 -0.359375zm0 5.796875l0.34375 0q0.515625 0 0.890625 0.375q0.375 0.359375 0.375 0.875q0 0.53125 -0.375 0.90625q-0.375 0.359375 -0.890625 0.359375l-0.34375 0q-0.515625 0 -0.890625 -0.359375q-0.359375 -0.375 -0.359375 -0.890625q0 -0.546875 0.375 -0.90625q0.375 -0.359375 0.875 -0.359375zm24.569336 -4.109375q1.078125 0.484375 1.640625 1.328125q0.578125 0.84375 0.578125 1.75q0 1.40625 -1.140625 2.484375q-1.140625 1.0625 -2.84375 1.0625q-1.0 0 -2.078125 -0
 .421875q-1.078125 -0.4375 -1.5 -0.875q-0.140625 -0.15625 -0.140625 -0.328125q0 -0.140625 0.09375 -0.25q0.109375 -0.109375 0.265625 -0.109375q0.140625 0 0.328125 0.140625q1.4375 1.0625 3.046875 1.0625q1.359375 0 2.265625 -0.84375q0.921875 -0.859375 0.921875 -1.890625q0 -0.703125 -0.453125 -1.359375q-0.453125 -0.671875 -1.296875 -1.0625q-0.84375 -0.390625 -1.703125 -0.390625q-0.28125 0 -0.40625 -0.09375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.109375 -0.109375 0.34375 -0.109375l0.671875 0q1.046875 0 1.71875 -0.640625q0.6875 -0.640625 0.6875 -1.484375q0 -0.875 -0.71875 -1.53125q-0.703125 -0.671875 -1.859375 -0.671875q-0.796875 0 -1.484375 0.28125q-0.671875 0.28125 -1.078125 0.75q-0.15625 0.1875 -0.234375 0.234375q-0.078125 0.046875 -0.171875 0.046875q-0.15625 0 -0.28125 -0.109375q-0.109375 -0.109375 -0.109375 -0.265625q0 -0.359375 0.8125 -0.9375q1.125 -0.78125 2.546875 -0.78125q1.484375 0 2.421875 0.890625q0.953125 0.875 0.953125 2.109375q0 0.78125 -0.4531
 25 1.484375q-0.453125 0.6875 -1.34375 1.09375z" fill-rule="nonzero"></path>
 <path fill="#000000" d="m240.9913 79.665405q-1.375 -1.75 -2.328125 -4.078125q-0.953125 -2.34375 -0.953125 -4.84375q0 -2.21875 0.703125 -4.234375q0.84375 -2.34375 2.578125 -4.671875l1.203125 0q-1.125 1.921875 -1.484375 2.75q-0.5625 1.28125 -0.890625 2.671875q-0.40625 1.734375 -0.40625 3.484375q0 4.46875 2.78125 8.921875l-1.203125 0zm3.275177 -4.0l0 -13.59375l6.03125 0q1.8125 0 2.75 0.359375q0.953125 0.359375 1.515625 1.296875q0.5625 0.921875 0.5625 2.046875q0 1.453125 -0.9375 2.453125q-0.921875 0.984375 -2.890625 1.25q0.71875 0.34375 1.09375 0.671875q0.78125 0.734375 1.484375 1.8125l2.375 3.703125l-2.265625 0l-1.796875 -2.828125q-0.796875 -1.21875 -1.3125 -1.875q-0.5 -0.65625 -0.90625 -0.90625q-0.40625 -0.265625 -0.8125 -0.359375q-0.3125 -0.078125 -1.015625 -0.078125l-2.078125 0l0 6.046875l-1.796875 0zm1.796875 -7.59375l3.859375 0q1.234375 0 1.921875 -0.25q0.703125 -0.265625 1.0625 -0.828125q0.375 -0.5625 0.375 -1.21875q0 -0.96875 -0.703125 -1.578125q-0.703125 -0.625 -2.21875 -0.625l
 -4.296875 0l0 4.5zm18.176086 4.421875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm15.547577 2.265625l1.640625 0.21875q-0.265625 1.6875 -1.375 2.65625q-1.109375 0.953125 -2.734375 0.953125q-2.015625 0 -3.25 -1.3125q-1.21875 -1.328125 -1.21875 -3.796875q0 -1.59375 0.515625 -2.78125q0.53125 -1.203125 1.609375 -1.796875q1.09375 -0.609375 2.359375 -0.609375q1.609375 0 2.625 0.8125q1.015625 0.8125 1.3125 2.3125l-1.625 0.25q-0.234375 -1
 .0 -0.828125 -1.5q-0.59375 -0.5 -1.421875 -0.5q-1.265625 0 -2.0625 0.90625q-0.78125 0.90625 -0.78125 2.859375q0 1.984375 0.765625 2.890625q0.765625 0.890625 1.984375 0.890625q0.984375 0 1.640625 -0.59375q0.65625 -0.609375 0.84375 -1.859375zm2.265625 -1.3125q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm9.281982 4.921875l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375
  3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm15.540802 0l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-
 0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm22.290802 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.110077 5.875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l
 0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm16.766357 0l0 -1.25q-0.9375 1.46875 -2.75 1.46875q-1.171875 0 -2.171875 -0.640625q-0.984375 -0.65625 -1.53125 -1.8125q-0.53125 -1.171875 -0.53125 -2.6875q0 -1.46875 0.484375 -2.671875q0.5 -1.203125 1.46875 -1.84375q0.984375 -0.640625 2.203125 -0.640625q0.890625 0 1.578125 0.375q0.703125 0.375 1.140625 0.984375l0 -4.875l1.65625 0l0 13.59375l-1.546875 0zm-5.28125 -4.921875q0 1.890625 0.796875 2.828125q0.8125 0.9375 1.890625 0.9375q1.09375 0 1.859375 -0.890625q0.765625 -0.890625 0.765625 -2.734375q0 -2.015625 -0.78125 -2.953125q-0.78125 -0.953125 -1.921875 -0.953125q-1.109375 0 -1.859375 0.90625q-0.75 0.90625 -0.75 2.859375zm16.016327 1.75l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2
 .484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm15.500732 5.875l0 -1.25q-0.9375 1.46875 -2.75 1.46875q-1.171875 0 -2.171875 -0.640625q-0.984375 -0.65625 -1.53125 -1.8125q-0.53125 -1.171875 -0.53125 -2.6875q0 -1.46875 0.484375 -2.671875q0.5 -1.203125 1.46875 -1.84375q0.984375 -0.640625 2.203125 -0.640625q0.890625 0 1.578125 0.375q0.703125 0.375 1.140625 0.984375l0 -4.875l1.65625 0l0 13.59375l-1.546875 0zm-5.28125 -4.921875q0 1.890625 0.796875 2.828125q0.8125 0.9375 1.890625 0.9375q1.09375 0 1.859375 -0.890625q0.765625 -0.890625 0.765625 -2.734375q0 -2.015625 -0.78125 -2.953125q-0.78125 -0
 .953125 -1.921875 -0.953125q-1.109375 0 -1.859375 0.90625q-0.75 0.90625 -0.75 2.859375zm17.184021 4.921875l-3.75 -9.859375l1.765625 0l2.125 5.90625q0.34375 0.953125 0.625 1.984375q0.21875 -0.78125 0.625 -1.875l2.1875 -6.015625l1.71875 0l-3.734375 9.859375l-1.5625 0zm13.03125 -1.21875q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.
 328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.047577 4.9375l0 -13.59375l1.671875 0l0 13.59375l-1.671875 0zm10.629211 0l0 -1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 -1.625 -0.328125q-0.75 -0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 -1.234375q-0.09375 -0.5 -0.09375 -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 1.3125 0.09375 1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 1.265625 0.375q0.75 0 1.40625 -0.375q0.65625 -0.390625 0.921875 -1.046875q0.28125 -0.671875 0.28125 -1.9375l0 -5.28125l1.671875 0l0 9.859375l-1.5 0zm10.672577 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.
 34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.578857 -2.078125l0 -1.90625l1.90625 0l0 1.90625l-1.90625 0zm0 7.953125l0 -1.90625l1.90625 0l0 1.90625l-1.90625 0zm10.100983 0l0 -13.59375l1.84375 0l7.140625 10.671875l0 -10.671875l1.71875 0l0 13.59375l-1.84375 0l-7.140625 -10.6875l0 10.6875l-1.71875 0zm19.72293 0l0 -1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 -1.625 -0.328125q-0.75 -0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 -1.234375q-0.09375 -0.5 -0.09375
  -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 1.3125 0.09375 1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 1.265625 0.375q0.75 0 1.40625 -0.375q0.65625 -0.390625 0.921875 -1.046875q0.28125 -0.671875 0.28125 -1.9375l0 -5.28125l1.671875 0l0 9.859375l-1.5 0zm3.9226074 0l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm17.087677 0l-1.546875 0l0 -13.59375l1.65625 0l0 4.84375q1.0
 625 -1.328125 2.703125 -1.328125q0.90625 0 1.71875 0.375q0.8125 0.359375 1.328125 1.03125q0.53125 0.65625 0.828125 1.59375q0.296875 0.9375 0.296875 2.0q0 2.53125 -1.25 3.921875q-1.25 1.375 -3.0 1.375q-1.75 0 -2.734375 -1.453125l0 1.234375zm-0.015625 -5.0q0 1.765625 0.46875 2.5625q0.796875 1.28125 2.140625 1.28125q1.09375 0 1.890625 -0.9375q0.796875 -0.953125 0.796875 -2.84375q0 -1.921875 -0.765625 -2.84375q-0.765625 -0.921875 -1.84375 -0.921875q-1.09375 0 -1.890625 0.953125q-0.796875 0.953125 -0.796875 2.75zm15.594452 1.828125l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5
  0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.094482 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0zm10.78656 -4.921875q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0
 .9375 -0.828125 2.828125zm9.688263 4.921875l0 -8.546875l-1.4844055 0l0 -1.3125l1.4844055 0l0 -1.046875q0 -0.984375 0.171875 -1.46875q0.234375 -0.65625 0.84375 -1.046875q0.609375 -0.40625 1.703125 -0.40625q0.703125 0 1.5625 0.15625l-0.25 1.46875q-0.515625 -0.09375 -0.984375 -0.09375q-0.765625 0 -1.078125 0.328125q-0.3125 0.3125 -0.3125 1.203125l0 0.90625l1.921875 0l0 1.3125l-1.921875 0l0 8.546875l-1.65625 0zm19.882202 -4.765625l1.796875 0.453125q-0.5625 2.21875 -2.03125 3.390625q-1.46875 1.15625 -3.59375 1.15625q-2.203125 0 -3.578125 -0.890625q-1.375 -0.90625 -2.09375 -2.59375q-0.71875 -1.703125 -0.71875 -3.65625q0 -2.125 0.796875 -3.703125q0.8125 -1.578125 2.3125 -2.390625q1.5 -0.828125 3.296875 -0.828125q2.046875 0 3.4375 1.046875q1.390625 1.03125 1.9375 2.90625l-1.765625 0.421875q-0.46875 -1.484375 -1.375 -2.15625q-0.90625 -0.6875 -2.265625 -0.6875q-1.5625 0 -2.625 0.75q-1.046875 0.75 -1.484375 2.03125q-0.421875 1.265625 -0.421875 2.609375q0 1.734375 0.5 3.03125q0.515625 1.28125 1
 .578125 1.921875q1.078125 0.640625 2.3125 0.640625q1.515625 0 2.5625 -0.859375q1.046875 -0.875 1.421875 -2.59375zm3.7698364 4.765625l0 -13.59375l5.125 0q1.359375 0 2.078125 0.125q1.0 0.171875 1.671875 0.640625q0.671875 0.46875 1.078125 1.3125q0.421875 0.84375 0.421875 1.84375q0 1.734375 -1.109375 2.9375q-1.09375 1.203125 -3.984375 1.203125l-3.484375 0l0 5.53125l-1.796875 0zm1.796875 -7.140625l3.515625 0q1.75 0 2.46875 -0.640625q0.734375 -0.65625 0.734375 -1.828125q0 -0.859375 -0.4375 -1.46875q-0.421875 -0.609375 -1.125 -0.796875q-0.453125 -0.125 -1.671875 -0.125l-3.484375 0l0 4.859375zm19.568542 -6.453125l1.796875 0l0 7.84375q0 2.0625 -0.46875 3.265625q-0.453125 1.203125 -1.671875 1.96875q-1.203125 0.75 -3.171875 0.75q-1.90625 0 -3.125 -0.65625q-1.21875 -0.65625 -1.734375 -1.90625q-0.515625 -1.25 -0.515625 -3.421875l0 -7.84375l1.796875 0l0 7.84375q0 1.765625 0.328125 2.609375q0.328125 0.84375 1.125 1.296875q0.8125 0.453125 1.96875 0.453125q1.984375 0 2.828125 -0.890625q0.84375 -0.90
 625 0.84375 -3.46875l0 -7.84375zm15.953125 9.984375l1.640625 0.21875q-0.265625 1.6875 -1.375 2.65625q-1.109375 0.953125 -2.734375 0.953125q-2.015625 0 -3.25 -1.3125q-1.21875 -1.328125 -1.21875 -3.796875q0 -1.59375 0.515625 -2.78125q0.53125 -1.203125 1.609375 -1.796875q1.09375 -0.609375 2.359375 -0.609375q1.609375 0 2.625 0.8125q1.015625 0.8125 1.3125 2.3125l-1.625 0.25q-0.234375 -1.0 -0.828125 -1.5q-0.59375 -0.5 -1.421875 -0.5q-1.265625 0 -2.0625 0.90625q-0.78125 0.90625 -0.78125 2.859375q0 1.984375 0.765625 2.890625q0.765625 0.890625 1.984375 0.890625q0.984375 0 1.640625 -0.59375q0.65625 -0.609375 0.84375 -1.859375zm2.265625 -1.3125q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25
  0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm9.266357 4.921875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0zm12.9782715 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-
 0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm8.438232 2.9375l1.65625 -0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 -0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 -0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 -0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 0.
 875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm11.09375 6.9375l-1.1875 0q2.765625 -4.453125 2.765625 -8.921875q0 -1.734375 -0.390625 -3.453125q-0.328125 -1.390625 -0.890625 -2.671875q-0.359375 -0.84375 -1.484375 -2.78125l1.1875 0q1.75 2.328125 2.578125 4.671875q0.71875 2.015625 0.71875 4.234375q0 2.5 -0.96875 4.84375q-0.953125 2.328125 -2.328125 4.078125z" fill-rule="nonzero"></path>
 <path fill="#cfe2f3" d="m196.45932 372.81628l242.11023 0l0 264.44092l-242.11023 0z" fill-rule="nonzero"></path>
@@ -509,7 +508,6 @@ under the License.
 <path fill="#000000" d="m29.261072 491.44757l-3.609375 -13.59375l1.84375 0l2.0625 8.90625q0.34375 1.40625 0.578125 2.78125q0.515625 -2.171875 0.609375 -2.515625l2.593752 -9.171875l2.171875 0l1.953125 6.875q0.734375 2.5625 1.046875 4.8125q0.265625 -1.28125 0.6875 -2.953125l2.125 -8.734375l1.8125 0l-3.734375 13.59375l-1.734375 0l-2.859375 -10.359375q-0.359375 -1.296875 -0.421875 -1.59375q-0.21875 0.9375 -0.40625 1.59375l-2.890627 10.359375l-1.828125 0zm15.01491 0l0 -13.59375l1.671875 0l0 4.875q1.171875 -1.359375 2.953125 -1.359375q1.09375 0 1.890625 0.4375q0.8125 0.421875 1.15625 1.1875q0.359375 0.765625 0.359375 2.203125l0 6.25l-1.671875 0l0 -6.25q0 -1.25 -0.546875 -1.8125q-0.546875 -0.578125 -1.53125 -0.578125q-0.75 0 -1.40625 0.390625q-0.640625 0.375 -0.921875 1.046875q-0.28125 0.65625 -0.28125 1.8125l0 5.390625l-1.671875 0zm17.125717 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265
 625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.110088 5.875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm15.559021 0l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.10937
 5 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm9.750717 -4.921875q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm20.902771 3.703125q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -
 0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875
 l0 -0.609375zm4.063217 4.9375l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0zm5.931427 0.8125l1.609375 0.25q0.109375 0.75 0.578125 1.09375q0.609375 0.453125 1.6875 0.453125q1.171875 0 1.7968826 -0.46875q0.625 -0.453125 0.859375 -1.28125q0.125 -0.515625 0.109375 -2.15625q-1.09375 1.296875 -2.7187576 1.296875q-2.03125 0 -3.15625 -1.46875q-1.109375 -1.46875 -1.109375 -3.515625q0 -1.40625 0.515625 -2.59375q0.515625 -1.203125 1.484375 -1.84375q0.96875 -0.65625 2.265625 -0.65625q1.7500076 0 2.8750076 1.40625l0 -1.1875l1.546875 0l0 8.515625q0 2.3125 -0.46875 3.265625q-0.46875 0.96875 -1.484375 1.515625q-1.015625 0.5625 -2.5000076 0.5625q-1.765625 0 -2.859375 -0.796875q-1.078125 -0.796875 -1.03125 -2.390625zm1.375 -5.921875q0 1.953125 0.
 765625 2.84375q0.78125 0.890625 1.9375 0.890625q1.1406326 0 1.9218826 -0.890625q0.78125 -0.890625 0.78125 -2.78125q0 -1.8125 -0.8125 -2.71875q-0.7968826 -0.921875 -1.9218826 -0.921875q-1.109375 0 -1.890625 0.90625q-0.78125 0.890625 -0.78125 2.671875zm15.750725 5.109375l0 -1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 -1.625 -0.328125q-0.75 -0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 -1.234375q-0.09375 -0.5 -0.09375 -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 1.3125 0.09375 1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 1.265625 0.375q0.75 0 1.40625 -0.375q0.65625 -0.390625 0.921875 -1.046875q0.28125 -0.671875 0.28125 -1.9375l0 -5.28125l1.671875 0l0 9.859375l-1.5 0zm3.9225922 0l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.20312
 5q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm22.290802 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.04
 6875zm9.110092 5.875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm14.031967 -1.5l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 -1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 -1.984375l0 -5.65625l-1.234375 0l0 -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 -0.078125zm6.4134827 2.3125l1.609375 0.25q0.109375 0.75 0.578125 1.09375q0.609375 0.453125 1.6875 0.453125q1.171875 0 1.796875 -0.46875q0.625 -0.453125 0.859375 -1.28125q0.125 -0.515625 0.
 109375 -2.15625q-1.09375 1.296875 -2.71875 1.296875q-2.03125 0 -3.15625 -1.46875q-1.109375 -1.46875 -1.109375 -3.515625q0 -1.40625 0.515625 -2.59375q0.515625 -1.203125 1.484375 -1.84375q0.96875 -0.65625 2.265625 -0.65625q1.75 0 2.875 1.40625l0 -1.1875l1.546875 0l0 8.515625q0 2.3125 -0.46875 3.265625q-0.46875 0.96875 -1.484375 1.515625q-1.015625 0.5625 -2.5 0.5625q-1.765625 0 -2.859375 -0.796875q-1.078125 -0.796875 -1.03125 -2.390625zm1.375 -5.921875q0 1.953125 0.765625 2.84375q0.78125 0.890625 1.9375 0.890625q1.140625 0 1.921875 -0.890625q0.78125 -0.890625 0.78125 -2.78125q0 -1.8125 -0.8125 -2.71875q-0.796875 -0.921875 -1.921875 -0.921875q-1.109375 0 -1.890625 0.90625q-0.78125 0.890625 -0.78125 2.671875zm9.313217 -6.578125l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 11.6875l0 -9.859375l1.671875 0l0 9.859375l-1.671875 0zm6.863571 0l-3.75 -9.859375l1.765625 0l2.125 5.90625q0.34375 0.953125 0.625 1.984375q0.21875 -0.78125 0.625 -1.875l2.1875 -6.015625l1.71875 0l-3.734375 9.859375l-1
 .5625 0zm13.34375 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.110092 5.875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0
 .671875 -0.765625 2.578125l0 5.375l-1.671875 0zm10.813217 0l0 -1.90625l1.90625 0l0 1.90625q0 1.046875 -0.375 1.6875q-0.375 0.65625 -1.171875 1.0l-0.46875 -0.71875q0.53125 -0.21875 0.78125 -0.671875q0.25 -0.453125 0.28125 -1.296875l-0.953125 0z" fill-rule="nonzero"></path>
 <path fill="#000000" d="m28.182947 505.41632l0 1.421875q0.6875 -0.84375 1.484375 -1.265625q0.8125 -0.4375 1.890625 -0.4375q1.1562519 0 2.125002 0.546875q0.96875 0.53125 1.5 1.5q0.546875 0.953125 0.546875 2.0q0 1.6875 -1.203125 2.890625q-1.203125 1.1875 -2.953127 1.1875q-2.09375 0 -3.390625 -1.703125l0 4.6875l1.890625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-3.71875 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.09375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.046875 0l0 -10.046875l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0zm6.750002 3.765625q0 -1.34375 -0.984375 -2.296875q-0.96875 -0.96875 -2.375002 -0.96875q-1.421875 0 -2.40625 0.96875q-0.984375 0.96875 -0.984375 2.296875q0 1.359375 0.984375 2.328125q0.984375 0.953125 2.40625 0.953125q1.3906269 0 2.3
 75002 -0.953125q0.984375 -0.96875 0.984375 -2.328125zm9.82373 4.265625l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.75q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.03125 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.484375 -1.78125 -0.484375q-0.90625 0 -2.546875 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4375 -0.390625 2.1875 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.703125 0 -2.65625 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.
 5625 0.484375 1.625 0.484375q1.0 0 1.859375 -0.40625q0.875 -0.40625 1.796875 -1.28125l0 -1.71875zm8.22998 -4.15625l0 1.96875q1.53125 -1.375 2.28125 -1.765625q0.75 -0.40625 1.390625 -0.40625q0.703125 0 1.296875 0.46875q0.59375 0.46875 0.59375 0.71875q0 0.171875 -0.109375 0.296875q-0.109375 0.109375 -0.296875 0.109375q-0.09375 0 -0.15625 -0.03125q-0.0625 -0.03125 -0.234375 -0.21875q-0.328125 -0.3125 -0.578125 -0.4375q-0.234375 -0.125 -0.46875 -0.125q-0.5 0 -1.21875 0.40625q-0.71875 0.40625 -2.5 2.0l0 4.265625l3.453125 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.109375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.265625q0 -0.15625 0.109375 -0.265625q0.109375 -0.109375 0.40625 -0.109375l1.875 0l0 -6.5l-1.4375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l2.21875 0zm14.573727 8.03125l0 -1.125q-1.6875 1.43
 75 -3.6249962 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.75q0 -1.140625 1.03125 -1.984375q1.046875 -0.84375 3.0312462 -0.84375q0.546875 0 1.171875 0.078125q0.625 0.0625 1.359375 0.21875l0 -1.265625q0 -0.640625 -0.59375 -1.109375q-0.59375 -0.484375 -1.78125 -0.484375q-0.90625 0 -2.5468712 0.53125q-0.296875 0.09375 -0.375 0.09375q-0.15625 0 -0.265625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.15625 0.09375 -0.25q0.140625 -0.140625 1.0625 -0.375q1.4374962 -0.390625 2.1874962 -0.390625q1.46875 0 2.296875 0.734375q0.828125 0.71875 0.828125 1.640625l0 5.15625l1.03125 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-1.828125 0zm0 -3.875q-0.546875 -0.15625 -1.15625 -0.234375q-0.609375 -0.078125 -1.28125 -0.078125q-1.7031212 0 -2.6562462 0.734375q-0.734375 0.546875 -0.734375 1.3125q0 0.703125 0.546875 1.1875q0.5625 0.484375 1.625 0.484375q0.9999962 0 1.8593712 -0.40625q0.875 -0.40625 1
 .796875 -1.28125l0 -1.71875zm9.558105 -7.765625l0 10.859375l3.0625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.90625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l3.0625 0l0 -10.078125l-2.234375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l3.015625 0zm11.401855 0l0 10.859375l3.0625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.90625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l3.0625 0l0 -10.078125l-2.234375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l3.015625 0zm15.22998 7.8125l-7.953125 0
 q0.21875 1.515625 1.28125 2.4375q1.0625 0.921875 2.625 0.921875q0.875 0 1.828125 -0.28125q0.953125 -0.296875 1.5625 -0.765625q0.171875 -0.140625 0.296875 -0.140625q0.15625 0 0.265625 0.125q0.109375 0.109375 0.109375 0.265625q0 0.15625 -0.15625 0.296875q-0.4375 0.46875 -1.578125 0.875q-1.140625 0.40625 -2.328125 0.40625q-2.0 0 -3.34375 -1.3125q-1.34375 -1.3125 -1.34375 -3.1875q0 -1.6875 1.25 -2.90625q1.265625 -1.21875 3.125 -1.21875q1.90625 0 3.140625 1.25q1.234375 1.25 1.21875 3.234375zm-0.796875 -0.796875q-0.234375 -1.28125 -1.21875 -2.09375q-0.984375 -0.8125 -2.34375 -0.8125q-1.375 0 -2.359375 0.8125q-0.96875 0.796875 -1.21875 2.09375l7.140625 0zm8.370605 -7.015625l0 10.859375l3.0625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-6.90625 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l3.0625 0l0 -10.078125l-2.234375 0q-0.281
 25 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.421875 -0.109375l3.015625 0zm11.401855 -0.390625l0 2.015625l-1.15625 0l0 -2.015625l1.15625 0zm0.015625 4.0l0 7.25l3.0625 0q0.296875 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.40625 0.109375l-6.90625 0q-0.265625 0 -0.390625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.390625 -0.109375l3.0625 0l0 -6.46875l-2.265625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l3.046875 0zm13.573738 0.5q0 -0.265625 0.109375 -0.375q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.40625l0 1.328125q0 0.28125 -0.109375 0.40625q-0.109375 0.109375 -0.28125 0.109375q-0.171875 0 -0.28125 -0.09375q-0.09375 -0.109375 -0.109375 -0.34375q-0.0625 -0.546875 -0.578125 -0.90625q-0.765625 -0.53125 -2.015625 
 -0.53125q-1.3125 0 -2.03125 0.53125q-0.546875 0.40625 -0.546875 0.890625q0 0.5625 0.640625 0.9375q0.453125 0.25 1.703125 0.390625q1.609375 0.171875 2.25 0.40625q0.890625 0.328125 1.328125 0.90625q0.453125 0.5625 0.453125 1.234375q0 0.984375 -0.953125 1.765625q-0.953125 0.78125 -2.8125 0.78125q-1.84375 0 -3.015625 -0.9375q0 0.3125 -0.046875 0.40625q-0.03125 0.09375 -0.125 0.15625q-0.09375 0.0625 -0.21875 0.0625q-0.171875 0 -0.28125 -0.109375q-0.109375 -0.125 -0.109375 -0.40625l0 -1.609375q0 -0.28125 0.109375 -0.390625q0.109375 -0.125 0.28125 -0.125q0.171875 0 0.28125 0.125q0.109375 0.109375 0.109375 0.296875q0 0.421875 0.21875 0.71875q0.328125 0.4375 1.03125 0.734375q0.71875 0.296875 1.75 0.296875q1.515625 0 2.25 -0.5625q0.75 -0.578125 0.75 -1.203125q0 -0.71875 -0.75 -1.15625q-0.765625 -0.4375 -2.21875 -0.578125q-1.453125 -0.15625 -2.078125 -0.390625q-0.625 -0.25 -0.984375 -0.734375q-0.359375 -0.484375 -0.359375 -1.03125q0 -1.015625 0.984375 -1.59375q0.984375 -0.59375 2.359375 -0.593
 75q1.609375 0 2.625 0.78125zm5.2768555 -0.5l0 0.78125q0.984375 -1.0625 1.984375 -1.0625q0.609375 0 1.0625 0.328125q0.453125 0.3125 0.765625 0.96875q0.515625 -0.65625 1.046875 -0.96875q0.53125 -0.328125 1.078125 -0.328125q0.84375 0 1.34375 0.546875q0.65625 0.703125 0.65625 1.546875l0 5.4375l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-1.4375 0l0 -6.15625q0 -0.59375 -0.359375 -0.984375q-0.359375 -0.390625 -0.828125 -0.390625q-0.4375 0 -0.90625 0.328125q-0.46875 0.3125 -1.078125 1.25l0 5.171875l0.65625 0q0.265625 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.390625 0.109375l-1.4375 0l0 -6.09375q0 -0.625 -0.375 -1.03125q-0.359375 -0.40625 -0.8125 -0.40625q-0.421875 0 -0.828125 0.28125q-0.5625 0.375 -1.1875 1.296875l0 5.171875l0.65625 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375
 l-2.09375 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l0.65625 0l0 -6.46875l-0.65625 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.4375 0zm14.776855 5.796875l0.34375 0q0.515625 0 0.890625 0.375q0.375 0.359375 0.375 0.875q0 0.53125 -0.375 0.90625q-0.375 0.359375 -0.890625 0.359375l-0.34375 0q-0.515625 0 -0.890625 -0.359375q-0.359375 -0.375 -0.359375 -0.890625q0 -0.546875 0.375 -0.90625q0.375 -0.359375 0.875 -0.359375zm15.38623 -9.40625l0 10.859375l1.046875 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-1.828125 0l0 -1.546875q-1.359375 1.859375 -3.453125 1.859375q-1.046875 0 -2.015625 -0.5625q-0.96875 -0.5625 -1.53125 -1.59375q-0.5625 -1.046875 -0.5625 -2.15625q0 -1.109375 0.5625 -2.140625q0.5625 -1.046875 1.53125 -1.609375q0.96875 -0.5625 2.03125 -0.562
 5q2.046875 0 3.4375 1.859375l0 -4.40625l-1.046875 0q-0.28125 0 -0.40625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.828125 0zm-0.78125 7.640625q0 -1.484375 -1.015625 -2.5q-1.0 -1.03125 -2.375 -1.03125q-1.390625 0 -2.390625 1.03125q-1.0 1.015625 -1.0 2.5q0 1.46875 1.0 2.5q1.0 1.03125 2.390625 1.03125q1.375 0 2.375 -1.03125q1.015625 -1.03125 1.015625 -2.5zm12.589355 0.171875l-7.953125 0q0.21875 1.515625 1.28125 2.4375q1.0625 0.921875 2.625 0.921875q0.875 0 1.828125 -0.28125q0.953125 -0.296875 1.5625 -0.765625q0.171875 -0.140625 0.296875 -0.140625q0.15625 0 0.265625 0.125q0.109375 0.109375 0.109375 0.265625q0 0.15625 -0.15625 0.296875q-0.4375 0.46875 -1.578125 0.875q-1.140625 0.40625 -2.328125 0.40625q-2.0 0 -3.34375 -1.3125q-1.34375 -1.3125 -1.34375 -3.1875q0 -1.6875 1.25 -2.90625q1.265625 -1.21875 3.125 -1.21875q1.90625 0 3.140625 1.25q1.234375 1.25 1.21875 3.234375zm-0.796875 -0.796875q-0.234375 -1.28125 -1.21875 -2.09375
 q-0.984375 -0.8125 -2.34375 -0.8125q-1.375 0 -2.359375 0.8125q-0.96875 0.796875 -1.21875 2.09375l7.140625 0zm7.4643555 -2.625l0 6.46875l3.421875 0q0.28125 0 0.390625 0.109375q0.125 0.109375 0.125 0.28125q0 0.171875 -0.125 0.28125q-0.109375 0.109375 -0.390625 0.109375l-6.09375 0q-0.28125 0 -0.40625 -0.109375q-0.109375 -0.109375 -0.109375 -0.28125q0 -0.171875 0.109375 -0.28125q0.125 -0.109375 0.40625 -0.109375l1.875 0l0 -6.46875l-1.6875 0q-0.265625 0 -0.390625 -0.109375q-0.125 -0.109375 -0.125 -0.28125q0 -0.171875 0.125 -0.28125q0.125 -0.109375 0.390625 -0.109375l1.6875 0l0 -1.1875q0 -0.984375 0.796875 -1.703125q0.8125 -0.71875 2.125 -0.71875q1.109375 0 2.359375 0.203125q0.46875 0.078125 0.5625 0.1875q0.09375 0.09375 0.09375 0.25q0 0.171875 -0.109375 0.28125q-0.109375 0.109375 -0.296875 0.109375q-0.0625 0 -0.25 -0.03125q-1.390625 -0.21875 -2.359375 -0.21875q-1.015625 0 -1.578125 0.5q-0.546875 0.5 -0.546875 1.140625l0 1.1875l3.640625 0q0.28125 0 0.40625 0.109375q0.125 0.109375 0.125 0.
 28125q0 0.171875 -0.125 0.28125q-0.125 0.109375 -0.40625 0.109375l-3.640625 0zm14.151855 7.25l0 -1.125q-1.6875 1.4375 -3.625 1.4375q-1.40625 0 -2.1875 -0.703125q-0.78125 -0.71875 -0.78125 -1.7

<TRUNCATED>